From aa4d87ffb9caad6adf6567002778e5394ae945a7 Mon Sep 17 00:00:00 2001 From: Socrates Date: Sat, 21 Mar 2026 08:07:11 +0800 Subject: [PATCH 1/3] [Refactor](Multi Catalog) Unify external meta cache framework (#60937) Part of #60686 This PR continues the external metadata cache refactor by introducing a unified engine-scoped cache framework and aligning multiple external catalog implementations on top of it. The goal is not to finish every historical cache migration in one shot. The goal is to make the framework shape explicit and consistent enough that later migrations can continue on top of one model instead of adding more engine-specific cache flows. At a high level, this PR does three things: - introduces a common framework for external metadata cache lifecycle, routing, invalidation, and stats - moves more engine-specific cache behavior behind engine adapters instead of scattered table- and util-level entry points - improves cross-engine consistency and test coverage while keeping legacy compatibility migration incremental The current framework can be viewed as three layers: 1. manager layer: owns engine cache lifecycle and routes cache operations 2. engine layer: each engine implements its own cache adapter on the shared framework 3. catalog/entry layer: each engine keeps per-catalog cache groups and typed cache entries ```mermaid flowchart TD A["ExternalMetaCacheMgr"] --> B["ExternalMetaCacheRegistry"] A --> C["ExternalMetaCacheRouteResolver"] A --> D["ExternalMetaCache(engine facade)"] D --> E["AbstractExternalMetaCache"] E --> F["CatalogEntryGroup(catalog scoped)"] F --> G["MetaCacheEntry(table/schema/partition/...)"] H["IcebergExternalMetaCache"] --> E I["PaimonExternalMetaCache"] --> E J["HudiExternalMetaCache"] --> E K["MaxComputeExternalMetaCache"] --> E L["DorisExternalMetaCache"] --> E M["HiveExternalMetaCache"] --> E ``` This structure makes a few framework boundaries explicit: - manager-level logic is responsible for engine registration, route resolution, and scoped invalidation dispatch - engine adapters own engine-specific metadata loading and cache composition - shared framework code owns per-catalog entry containers, generic entry access, and common lifecycle behavior - add a shared external meta cache framework under `datasource.metacache` - refactor `ExternalMetaCacheMgr` so registration and routing are more explicit instead of staying mixed in one manager path - make cache initialization and invalidation flow clearer at the framework level - align multiple engines with the framework model, including Iceberg, Paimon, Hudi, MaxCompute, Doris, and Hive - keep legacy compatibility migration incremental instead of forcing a one-PR full replacement of every historical cache path - add or extend framework-level and engine-level tests around routing, invalidation, and cache behavior This PR is mainly about pulling different engines closer to one framework shape. - Iceberg and Paimon are aligned with the framework while keeping latest snapshot metadata modeled as table-owned runtime projection - Hudi moves further away from ad hoc cache state and closer to framework-owned entry behavior - MaxCompute and Doris move more cache ownership into engine adapters - Hive keeps its existing complexity where necessary, but more of that logic now sits behind the framework-oriented cache layer The important point is not that every engine is identical now. The important point is that they are being moved toward one consistent framework model. - this is primarily a framework refactor and behavior-alignment change - migration is still incremental, so some legacy compatibility paths are intentionally retained - the purpose of this PR is to reduce structural divergence across engines without requiring a full one-shot migration (cherry picked from commit 010f4706909b5ade953242d333f897fe4ffd1e65) --- ...chema_catalog_meta_cache_stats_scanner.cpp | 25 +- be/src/information_schema/schema_scanner.cpp | 10 + .../apache/doris/catalog/RefreshManager.java | 12 +- .../org/apache/doris/catalog/SchemaTable.java | 27 +- .../apache/doris/common/util/BrokerUtil.java | 4 +- .../apache/doris/datasource/CatalogMgr.java | 148 ++- .../doris/datasource/ExternalCatalog.java | 10 +- .../doris/datasource/ExternalDatabase.java | 4 +- .../datasource/ExternalMetaCacheMgr.java | 456 ++++--- .../doris/datasource/ExternalSchemaCache.java | 151 --- .../doris/datasource/ExternalTable.java | 15 +- .../apache/doris/datasource/NameMapping.java | 5 + ...ifestCacheKey.java => SchemaCacheKey.java} | 35 +- .../doris/datasource/SchemaCacheValue.java | 2 +- .../doris/DorisExternalMetaCache.java | 133 ++ .../doris/DorisExternalMetaCacheMgr.java | 83 -- .../doris/RemoteDorisExternalTable.java | 5 + .../datasource/doris/RemoteOlapTable.java | 11 +- .../doris/datasource/hive/AcidUtil.java | 2 +- .../datasource/hive/HMSExternalCatalog.java | 9 +- .../datasource/hive/HMSExternalTable.java | 106 +- .../doris/datasource/hive/HiveDlaTable.java | 18 +- .../hive/HiveExternalMetaCache.java | 1095 +++++++++++++++++ .../datasource/hive/HiveMetaStoreCache.java | 1064 ---------------- .../doris/datasource/hive/HudiDlaTable.java | 7 +- .../hive/event/MetastoreEventsProcessor.java | 2 +- .../datasource/hive/source/HiveScanNode.java | 42 +- .../hudi/HudiExternalMetaCache.java | 238 ++++ .../datasource/hudi/HudiFsViewCacheKey.java | 58 + .../hudi/HudiMetaClientCacheKey.java | 58 + .../hudi/HudiPartitionCacheKey.java | 72 ++ .../datasource/hudi/HudiPartitionUtils.java | 90 ++ .../datasource/hudi/HudiSchemaCacheKey.java | 2 +- .../doris/datasource/hudi/HudiUtils.java | 18 +- .../hudi/source/COWIncrementalRelation.java | 3 +- .../source/HudiCachedFsViewProcessor.java | 136 -- .../source/HudiCachedMetaClientProcessor.java | 147 --- .../source/HudiCachedPartitionProcessor.java | 193 --- .../hudi/source/HudiMetadataCacheMgr.java | 145 --- .../hudi/source/HudiPartitionProcessor.java | 132 -- .../datasource/hudi/source/HudiScanNode.java | 7 +- .../iceberg/IcebergExternalCatalog.java | 15 +- .../iceberg/IcebergExternalMetaCache.java | 289 +++++ .../iceberg/IcebergExternalTable.java | 15 +- .../iceberg/IcebergManifestEntryKey.java | 69 ++ .../iceberg/IcebergMetadataCache.java | 309 ----- .../iceberg/IcebergSchemaCacheKey.java | 2 +- .../iceberg/IcebergTableCacheValue.java | 20 +- .../datasource/iceberg/IcebergUtils.java | 49 +- .../iceberg/cache/IcebergManifestCache.java | 84 -- .../cache/IcebergManifestCacheLoader.java | 92 +- .../iceberg/source/IcebergApiSource.java | 4 +- .../iceberg/source/IcebergHMSSource.java | 5 +- .../iceberg/source/IcebergScanNode.java | 16 +- .../lakesoul/LakeSoulExternalCatalog.java | 1 - .../maxcompute/MaxComputeCacheKey.java | 65 - .../MaxComputeExternalMetaCache.java | 115 ++ .../maxcompute/MaxComputeExternalTable.java | 31 +- .../maxcompute/MaxComputeMetadataCache.java | 61 - .../MaxComputeMetadataCacheMgr.java | 69 -- .../metacache/AbstractExternalMetaCache.java | 334 +++++ .../doris/datasource/metacache/CacheSpec.java | 181 ++- .../CatalogEntryGroup.java} | 32 +- .../metacache/ExternalMetaCache.java | 120 ++ .../metacache/ExternalMetaCacheRegistry.java | 99 ++ .../ExternalMetaCacheRouteResolver.java | 93 ++ .../metacache/LegacyMetaCacheFactory.java | 49 + .../datasource/metacache/MetaCacheEntry.java | 188 +++ .../metacache/MetaCacheEntryDef.java | 235 ++++ .../metacache/MetaCacheEntryInvalidation.java | 96 ++ .../metacache/MetaCacheEntryStats.java | 189 +++ .../PaimonLatestSnapshotProjectionLoader.java | 83 ++ .../paimon/PaimonPartitionInfoLoader.java | 58 + .../metacache/paimon/PaimonTableLoader.java | 48 + .../paimon/PaimonExternalCatalog.java | 11 +- .../paimon/PaimonExternalMetaCache.java | 116 ++ .../paimon/PaimonExternalTable.java | 12 +- .../paimon/PaimonMetadataCache.java | 229 ---- .../paimon/PaimonSchemaCacheKey.java | 2 +- .../paimon/PaimonSysExternalTable.java | 12 +- .../paimon/PaimonTableCacheValue.java | 31 +- .../doris/datasource/paimon/PaimonUtils.java | 15 +- .../paimon/source/PaimonScanNode.java | 3 +- .../metastore/AbstractIcebergProperties.java | 15 +- .../datasource/test/TestExternalCatalog.java | 1 - .../doris/fs/remote/SwitchingFileSystem.java | 1 - .../rules/SortedPartitionRanges.java | 2 +- .../commands/insert/HiveInsertExecutor.java | 7 +- .../apache/doris/planner/HiveTableSink.java | 8 +- .../doris/statistics/HMSAnalysisTask.java | 5 +- .../tablefunction/MetadataGenerator.java | 115 +- .../ExternalMetaCacheRouteResolverTest.java | 376 ++++++ ...xternalTableSchemaCacheDelegationTest.java | 57 + .../doris/datasource/PathVisibleTest.java | 2 +- .../doris/DorisExternalMetaCacheTest.java | 62 + .../doris/datasource/hive/HiveAcidTest.java | 2 +- .../hive/HiveMetaStoreCacheTest.java | 82 +- .../hive/source/HiveScanNodeTest.java | 14 +- .../hudi/HudiExternalMetaCacheTest.java | 188 +++ .../doris/datasource/hudi/HudiUtilsTest.java | 74 +- .../iceberg/IcebergExternalMetaCacheTest.java | 373 ++++++ .../MaxComputeExternalMetaCacheTest.java | 111 ++ .../AbstractExternalMetaCacheTest.java | 138 +++ .../datasource/metacache/CacheSpecTest.java | 169 +++ .../metacache/MetaCacheEntryTest.java | 199 +++ .../paimon/PaimonExternalMetaCacheTest.java | 123 ++ .../doris/external/hms/HmsCatalogTest.java | 18 +- .../doris/load/loadv2/LoadManagerTest.java | 7 +- .../translator/CountStarSmallestSlotTest.java | 13 +- .../planner/ListPartitionPrunerV2Test.java | 69 +- .../apache/doris/qe/HmsQueryCacheTest.java | 8 +- .../hive/test_external_catalog_hive.groovy | 13 +- .../iceberg/test_iceberg_table_stats.groovy | 14 +- 113 files changed, 6973 insertions(+), 3620 deletions(-) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalSchemaCache.java rename fe/fe-core/src/main/java/org/apache/doris/datasource/{iceberg/cache/ManifestCacheKey.java => SchemaCacheKey.java} (58%) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/doris/DorisExternalMetaCache.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/doris/DorisExternalMetaCacheMgr.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveExternalMetaCache.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiExternalMetaCache.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiFsViewCacheKey.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiMetaClientCacheKey.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiPartitionCacheKey.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiPartitionUtils.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedFsViewProcessor.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedMetaClientProcessor.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedPartitionProcessor.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiMetadataCacheMgr.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiPartitionProcessor.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalMetaCache.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergManifestEntryKey.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/cache/IcebergManifestCache.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeCacheKey.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalMetaCache.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeMetadataCache.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeMetadataCacheMgr.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/AbstractExternalMetaCache.java rename fe/fe-core/src/main/java/org/apache/doris/datasource/{CatalogScopedCacheMgr.java => metacache/CatalogEntryGroup.java} (50%) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/ExternalMetaCache.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/ExternalMetaCacheRegistry.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/ExternalMetaCacheRouteResolver.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/LegacyMetaCacheFactory.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCacheEntry.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCacheEntryDef.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCacheEntryInvalidation.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCacheEntryStats.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/paimon/PaimonLatestSnapshotProjectionLoader.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/paimon/PaimonPartitionInfoLoader.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/paimon/PaimonTableLoader.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalMetaCache.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCache.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/ExternalMetaCacheRouteResolverTest.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/ExternalTableSchemaCacheDelegationTest.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/doris/DorisExternalMetaCacheTest.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/hudi/HudiExternalMetaCacheTest.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergExternalMetaCacheTest.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalMetaCacheTest.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/metacache/AbstractExternalMetaCacheTest.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/metacache/CacheSpecTest.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/metacache/MetaCacheEntryTest.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/paimon/PaimonExternalMetaCacheTest.java diff --git a/be/src/information_schema/schema_catalog_meta_cache_stats_scanner.cpp b/be/src/information_schema/schema_catalog_meta_cache_stats_scanner.cpp index e94448a1329895..97a169435106c6 100644 --- a/be/src/information_schema/schema_catalog_meta_cache_stats_scanner.cpp +++ b/be/src/information_schema/schema_catalog_meta_cache_stats_scanner.cpp @@ -30,10 +30,29 @@ namespace doris { #include "common/compile_check_begin.h" std::vector SchemaCatalogMetaCacheStatsScanner::_s_tbls_columns = { + {"FE_HOST", TYPE_STRING, sizeof(StringRef), true}, {"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}, + {"ENGINE_NAME", TYPE_STRING, sizeof(StringRef), true}, + {"ENTRY_NAME", TYPE_STRING, sizeof(StringRef), true}, + {"EFFECTIVE_ENABLED", TYPE_BOOLEAN, sizeof(bool), true}, + {"CONFIG_ENABLED", TYPE_BOOLEAN, sizeof(bool), true}, + {"AUTO_REFRESH", TYPE_BOOLEAN, sizeof(bool), true}, + {"TTL_SECOND", TYPE_BIGINT, sizeof(int64_t), true}, + {"CAPACITY", TYPE_BIGINT, sizeof(int64_t), true}, + {"ESTIMATED_SIZE", TYPE_BIGINT, sizeof(int64_t), true}, + {"REQUEST_COUNT", TYPE_BIGINT, sizeof(int64_t), true}, + {"HIT_COUNT", TYPE_BIGINT, sizeof(int64_t), true}, + {"MISS_COUNT", TYPE_BIGINT, sizeof(int64_t), true}, + {"HIT_RATE", TYPE_DOUBLE, sizeof(double), true}, + {"LOAD_SUCCESS_COUNT", TYPE_BIGINT, sizeof(int64_t), true}, + {"LOAD_FAILURE_COUNT", TYPE_BIGINT, sizeof(int64_t), true}, + {"TOTAL_LOAD_TIME_MS", TYPE_BIGINT, sizeof(int64_t), true}, + {"AVG_LOAD_PENALTY_MS", TYPE_DOUBLE, sizeof(double), true}, + {"EVICTION_COUNT", TYPE_BIGINT, sizeof(int64_t), true}, + {"INVALIDATE_COUNT", TYPE_BIGINT, sizeof(int64_t), true}, + {"LAST_LOAD_SUCCESS_TIME", TYPE_STRING, sizeof(StringRef), true}, + {"LAST_LOAD_FAILURE_TIME", TYPE_STRING, sizeof(StringRef), true}, + {"LAST_ERROR", TYPE_STRING, sizeof(StringRef), true}, }; SchemaCatalogMetaCacheStatsScanner::SchemaCatalogMetaCacheStatsScanner() diff --git a/be/src/information_schema/schema_scanner.cpp b/be/src/information_schema/schema_scanner.cpp index 5c0dfa59c21017..d9416769fa6ba2 100644 --- a/be/src/information_schema/schema_scanner.cpp +++ b/be/src/information_schema/schema_scanner.cpp @@ -456,6 +456,16 @@ Status SchemaScanner::insert_block_column(TCell cell, int col_index, Block* bloc break; } + case TYPE_FLOAT: { + assert_cast(col_ptr)->insert_value(cell.doubleVal); + break; + } + + case TYPE_DOUBLE: { + assert_cast(col_ptr)->insert_value(cell.doubleVal); + break; + } + case TYPE_BOOLEAN: { reinterpret_cast(col_ptr)->insert_value(cell.boolVal); break; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/RefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/RefreshManager.java index 4ee6a2cab07a88..86b664eaf36078 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/RefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/RefreshManager.java @@ -28,7 +28,7 @@ import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalTable; -import org.apache.doris.datasource.hive.HiveMetaStoreCache; +import org.apache.doris.datasource.hive.HiveExternalMetaCache; import org.apache.doris.datasource.iceberg.IcebergExternalTable; import org.apache.doris.persist.OperationType; @@ -192,8 +192,8 @@ public void replayRefreshTable(ExternalObjectLog log) { && ((modifiedPartNames != null && !modifiedPartNames.isEmpty()) || (newPartNames != null && !newPartNames.isEmpty()))) { // Partition-level cache invalidation, only for hive catalog - HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getMetaStoreCache((HMSExternalCatalog) catalog); + HiveExternalMetaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() + .hive(catalog.getId()); cache.refreshAffectedPartitionsCache((HMSExternalTable) table.get(), modifiedPartNames, newPartNames); if (table.get() instanceof HMSExternalTable && log.getLastUpdateTime() > 0) { ((HMSExternalTable) table.get()).setUpdateTime(log.getLastUpdateTime()); @@ -276,7 +276,11 @@ public void refreshPartitions(String catalogName, String dbName, String tableNam return; } - Env.getCurrentEnv().getExtMetaCacheMgr().invalidatePartitionsCache((ExternalTable) table, partitionNames); + ExternalTable externalTable = (ExternalTable) table; + HiveExternalMetaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().hive(externalTable.getCatalog().getId()); + for (String partitionName : partitionNames) { + cache.invalidatePartitionCache(externalTable, partitionName); + } ((HMSExternalTable) table).setUpdateTime(updateTime); } 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 faae2c75098f21..6918a205ef13f0 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 @@ -612,10 +612,29 @@ public class SchemaTable extends Table { ) .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()) + builder().column("FE_HOST", ScalarType.createStringType()) + .column("CATALOG_NAME", ScalarType.createStringType()) + .column("ENGINE_NAME", ScalarType.createStringType()) + .column("ENTRY_NAME", ScalarType.createStringType()) + .column("EFFECTIVE_ENABLED", ScalarType.createType(PrimitiveType.BOOLEAN)) + .column("CONFIG_ENABLED", ScalarType.createType(PrimitiveType.BOOLEAN)) + .column("AUTO_REFRESH", ScalarType.createType(PrimitiveType.BOOLEAN)) + .column("TTL_SECOND", ScalarType.createType(PrimitiveType.BIGINT)) + .column("CAPACITY", ScalarType.createType(PrimitiveType.BIGINT)) + .column("ESTIMATED_SIZE", ScalarType.createType(PrimitiveType.BIGINT)) + .column("REQUEST_COUNT", ScalarType.createType(PrimitiveType.BIGINT)) + .column("HIT_COUNT", ScalarType.createType(PrimitiveType.BIGINT)) + .column("MISS_COUNT", ScalarType.createType(PrimitiveType.BIGINT)) + .column("HIT_RATE", ScalarType.createType(PrimitiveType.DOUBLE)) + .column("LOAD_SUCCESS_COUNT", ScalarType.createType(PrimitiveType.BIGINT)) + .column("LOAD_FAILURE_COUNT", ScalarType.createType(PrimitiveType.BIGINT)) + .column("TOTAL_LOAD_TIME_MS", ScalarType.createType(PrimitiveType.BIGINT)) + .column("AVG_LOAD_PENALTY_MS", ScalarType.createType(PrimitiveType.DOUBLE)) + .column("EVICTION_COUNT", ScalarType.createType(PrimitiveType.BIGINT)) + .column("INVALIDATE_COUNT", ScalarType.createType(PrimitiveType.BIGINT)) + .column("LAST_LOAD_SUCCESS_TIME", ScalarType.createStringType()) + .column("LAST_LOAD_FAILURE_TIME", ScalarType.createStringType()) + .column("LAST_ERROR", ScalarType.createStringType()) .build()) ) .put("backend_kerberos_ticket_cache", diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/BrokerUtil.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/BrokerUtil.java index e5b7c1a7f45376..452cc3296e859b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/BrokerUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/BrokerUtil.java @@ -27,7 +27,7 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; -import org.apache.doris.datasource.hive.HiveMetaStoreCache; +import org.apache.doris.datasource.hive.HiveExternalMetaCache; import org.apache.doris.fs.FileSystemFactory; import org.apache.doris.fs.remote.RemoteFile; import org.apache.doris.fs.remote.RemoteFileSystem; @@ -177,7 +177,7 @@ public static List parseColumnsFromPath( if (index == -1) { continue; } - columns[index] = HiveMetaStoreCache.HIVE_DEFAULT_PARTITION.equals(pair[1]) + columns[index] = HiveExternalMetaCache.HIVE_DEFAULT_PARTITION.equals(pair[1]) ? FeConstants.null_string : pair[1]; size++; if (size >= columnsFromPath.size()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java index 16eeebd7dffab8..9212761ebcd26f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java @@ -23,6 +23,7 @@ import org.apache.doris.catalog.EnvFactory; import org.apache.doris.catalog.TableIf; import org.apache.doris.cluster.ClusterNamespace; +import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.CaseSensibility; import org.apache.doris.common.DdlException; @@ -41,7 +42,10 @@ import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalDatabase; import org.apache.doris.datasource.hive.HMSExternalTable; +import org.apache.doris.datasource.hive.HiveExternalMetaCache; +import org.apache.doris.datasource.mvcc.MvccUtil; import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.exceptions.NotSupportedException; import org.apache.doris.nereids.trees.plans.commands.CreateCatalogCommand; import org.apache.doris.persist.OperationType; import org.apache.doris.persist.gson.GsonPostProcessable; @@ -121,20 +125,44 @@ private void addCatalog(CatalogIf catalog) { } } - private CatalogIf removeCatalog(long catalogId) { - CatalogIf catalog = idToCatalog.remove(catalogId); - LOG.info("Removed catalog with id {}, name {}", catalogId, catalog == null ? "N/A" : catalog.getName()); - if (catalog != null) { - Env.getCurrentEnv().getRefreshManager().removeFromRefreshMap(catalogId); - catalog.onClose(); - nameToCatalog.remove(catalog.getName()); - if (ConnectContext.get() != null) { - ConnectContext.get().removeLastDBOfCatalog(catalog.getName()); - } - Env.getCurrentEnv().getExtMetaCacheMgr().removeCache(catalog.getId()); - Env.getCurrentEnv().getQueryStats().clear(catalog.getId()); + private RemovedCatalog removeCatalog(long catalogId) { + CatalogIf catalog = idToCatalog.get(catalogId); + if (catalog == null) { + return null; + } + String catalogName = catalog.getName(); + Env.getCurrentEnv().getRefreshManager().removeFromRefreshMap(catalogId); + idToCatalog.remove(catalogId); + nameToCatalog.remove(catalogName); + return new RemovedCatalog(catalog, catalogName); + } + + private void cleanupRemovedCatalog(RemovedCatalog removedCatalog) { + if (removedCatalog == null) { + return; + } + CatalogIf catalog = removedCatalog.catalog; + catalog.onClose(); + Env.getCurrentEnv().getConstraintManager().dropCatalogConstraints(removedCatalog.catalogName); + ConnectContext ctx = ConnectContext.get(); + if (ctx != null) { + ctx.removeLastDBOfCatalog(removedCatalog.catalogName); + } + Env.getCurrentEnv().getExtMetaCacheMgr().removeCatalog(removedCatalog.catalogId); + Env.getCurrentEnv().getQueryStats().clear(removedCatalog.catalogId); + LOG.info("Removed catalog with id {}, name {}", removedCatalog.catalogId, removedCatalog.catalogName); + } + + private static final class RemovedCatalog { + private final CatalogIf catalog; + private final String catalogName; + private final long catalogId; + + private RemovedCatalog(CatalogIf catalog, String catalogName) { + this.catalog = catalog; + this.catalogName = catalogName; + this.catalogId = catalog.getId(); } - return catalog; } public InternalCatalog getInternalCatalog() { @@ -255,6 +283,7 @@ public void createCatalog(CreateCatalogCommand cmd) throws UserException { * Remove the catalog instance by name and write the meta log. */ public void dropCatalog(String catalogName, boolean ifExists) throws UserException { + RemovedCatalog removedCatalog = null; writeLock(); try { if (ifExists && !nameToCatalog.containsKey(catalogName)) { @@ -267,23 +296,24 @@ public void dropCatalog(String catalogName, boolean ifExists) throws UserExcepti } CatalogLog log = new CatalogLog(); log.setCatalogId(catalog.getId()); - replayDropCatalog(log); + removedCatalog = removeCatalog(log.getCatalogId()); Env.getCurrentEnv().getEditLog().logCatalogLog(OperationType.OP_DROP_CATALOG, log); - - if (ConnectContext.get() != null) { - ConnectContext.get().removeLastDBOfCatalog(catalogName); - } - Env.getCurrentEnv().getQueryStats().clear(catalog.getId()); - LOG.info("finished to drop catalog {}:{}", catalog.getName(), catalog.getId()); } finally { writeUnlock(); + cleanupRemovedCatalog(removedCatalog); } + if (removedCatalog == null) { + return; + } + LOG.info("finished to drop catalog {}:{}", removedCatalog.catalogName, removedCatalog.catalogId); } /** * Modify the catalog name into a new one and write the meta log. */ public void alterCatalogName(String catalogName, String newCatalogName) throws UserException { + RemovedCatalog removedCatalog = null; + String lastDb = null; writeLock(); try { CatalogIf catalog = nameToCatalog.get(catalogName); @@ -296,17 +326,49 @@ public void alterCatalogName(String catalogName, String newCatalogName) throws U CatalogLog log = new CatalogLog(); log.setCatalogId(catalog.getId()); log.setNewCatalogName(newCatalogName); - replayAlterCatalogName(log); - Env.getCurrentEnv().getEditLog().logCatalogLog(OperationType.OP_ALTER_CATALOG_NAME, log); + ConnectContext ctx = ConnectContext.get(); + if (ctx != null) { + lastDb = ctx.getLastDBOfCatalog(catalogName); + } + removedCatalog = removeCatalog(log.getCatalogId()); + } finally { + writeUnlock(); + } + cleanupRemovedCatalog(removedCatalog); + if (removedCatalog == null) { + throw new IllegalStateException("No catalog found with name: " + catalogName); + } + + writeLock(); + try { + DdlException ddlException = null; + CatalogIf catalog = removedCatalog.catalog; + if (nameToCatalog.get(newCatalogName) != null) { + addCatalog(catalog); + ddlException = new DdlException("Catalog with name " + newCatalogName + " already exist"); + } else { + catalog.modifyCatalogName(newCatalogName); + addCatalog(catalog); + + CatalogLog log = new CatalogLog(); + log.setCatalogId(catalog.getId()); + log.setNewCatalogName(newCatalogName); + Env.getCurrentEnv().getEditLog().logCatalogLog(OperationType.OP_ALTER_CATALOG_NAME, log); + } ConnectContext ctx = ConnectContext.get(); if (ctx != null) { - String db = ctx.getLastDBOfCatalog(catalogName); - if (db != null) { - ctx.removeLastDBOfCatalog(catalogName); - ctx.addLastDBOfCatalog(log.getNewCatalogName(), db); + if (lastDb != null) { + if (ddlException == null) { + ctx.addLastDBOfCatalog(newCatalogName, lastDb); + } else { + ctx.addLastDBOfCatalog(catalogName, lastDb); + } } } + if (ddlException != null) { + throw ddlException; + } } finally { writeUnlock(); } @@ -498,22 +560,37 @@ private void createCatalogInternal(CatalogIf catalog, boolean isReplay) throws D * Reply for drop catalog event. */ public void replayDropCatalog(CatalogLog log) { + RemovedCatalog removedCatalog; writeLock(); try { - removeCatalog(log.getCatalogId()); + removedCatalog = removeCatalog(log.getCatalogId()); } finally { writeUnlock(); } + cleanupRemovedCatalog(removedCatalog); } /** * Reply for alter catalog name event. */ public void replayAlterCatalogName(CatalogLog log) { + RemovedCatalog removedCatalog; + writeLock(); + try { + removedCatalog = removeCatalog(log.getCatalogId()); + } finally { + writeUnlock(); + } + cleanupRemovedCatalog(removedCatalog); + + if (removedCatalog == null) { + throw new IllegalStateException("No catalog found with id: " + log.getCatalogId()); + } + CatalogIf catalog = removedCatalog.catalog; + catalog.modifyCatalogName(log.getNewCatalogName()); + writeLock(); try { - CatalogIf catalog = removeCatalog(log.getCatalogId()); - catalog.modifyCatalogName(log.getNewCatalogName()); addCatalog(catalog); } finally { writeUnlock(); @@ -725,7 +802,15 @@ public void addExternalPartitions(String catalogName, String dbName, String tabl } HMSExternalTable hmsTable = (HMSExternalTable) table; - Env.getCurrentEnv().getExtMetaCacheMgr().addPartitionsCache(catalog.getId(), hmsTable, partitionNames); + List partitionColumnTypes; + try { + partitionColumnTypes = hmsTable.getPartitionColumnTypes(MvccUtil.getSnapshotFromContext(hmsTable)); + } catch (NotSupportedException e) { + LOG.warn("Ignore not supported hms table, message: {} ", e.getMessage()); + return; + } + HiveExternalMetaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().hive(catalog.getId()); + cache.addPartitionsCache(hmsTable.getOrBuildNameMapping(), partitionNames, partitionColumnTypes); hmsTable.setUpdateTime(updateTime); } @@ -756,7 +841,8 @@ public void dropExternalPartitions(String catalogName, String dbName, String tab } HMSExternalTable hmsTable = (HMSExternalTable) table; - Env.getCurrentEnv().getExtMetaCacheMgr().dropPartitionsCache(catalog.getId(), hmsTable, partitionNames); + Env.getCurrentEnv().getExtMetaCacheMgr().hive(catalog.getId()) + .dropPartitionsCache(hmsTable, partitionNames, true); hmsTable.setUpdateTime(updateTime); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java index d9dfe2822067bf..2689f5c962af02 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java @@ -35,7 +35,6 @@ import org.apache.doris.common.Version; import org.apache.doris.common.security.authentication.ExecutionAuthenticator; import org.apache.doris.common.util.Util; -import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey; import org.apache.doris.datasource.connectivity.CatalogConnectivityTestCoordinator; import org.apache.doris.datasource.doris.RemoteDorisExternalDatabase; import org.apache.doris.datasource.es.EsExternalDatabase; @@ -397,7 +396,7 @@ private void buildMetaCache() { if (LOG.isDebugEnabled()) { LOG.debug("buildMetaCache for catalog: {}:{}", this.name, this.id, new Exception()); } - metaCache = Env.getCurrentEnv().getExtMetaCacheMgr().buildMetaCache( + metaCache = Env.getCurrentEnv().getExtMetaCacheMgr().legacyMetaCacheFactory().build( name, OptionalLong.of(Config.external_cache_expire_time_seconds_after_access), OptionalLong.of(Config.external_cache_refresh_time_minutes * 60L), @@ -598,7 +597,7 @@ public void onRefreshCache(boolean invalidCache) { setLastUpdateTime(System.currentTimeMillis()); refreshMetaCacheOnly(); if (invalidCache) { - Env.getCurrentEnv().getExtMetaCacheMgr().invalidateCatalogCache(id); + Env.getCurrentEnv().getExtMetaCacheMgr().invalidateCatalog(id); } } @@ -1114,7 +1113,7 @@ public void unregisterDatabase(String dbName) { if (isInitialized()) { metaCache.invalidate(dbName, Util.genIdByName(name, dbName)); } - Env.getCurrentEnv().getExtMetaCacheMgr().invalidateDbCache(getId(), dbName); + Env.getCurrentEnv().getExtMetaCacheMgr().invalidateDb(getId(), dbName); } public void registerDatabase(long dbId, String dbName) { @@ -1334,7 +1333,8 @@ public void notifyPropertiesUpdated(Map updatedProps) { CatalogIf.super.notifyPropertiesUpdated(updatedProps); String schemaCacheTtl = updatedProps.getOrDefault(SCHEMA_CACHE_TTL_SECOND, null); if (java.util.Objects.nonNull(schemaCacheTtl)) { - Env.getCurrentEnv().getExtMetaCacheMgr().invalidSchemaCache(id); + ExternalMetaCacheMgr extMetaCacheMgr = Env.getCurrentEnv().getExtMetaCacheMgr(); + extMetaCacheMgr.removeCatalog(id); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalDatabase.java index 2c20daec0bfe1d..1d4c17465036ca 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalDatabase.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalDatabase.java @@ -127,7 +127,7 @@ public void resetMetaToUninitialized() { metaCache.invalidateAll(); } } - Env.getCurrentEnv().getExtMetaCacheMgr().invalidateDbCache(getCatalog().getId(), getFullName()); + Env.getCurrentEnv().getExtMetaCacheMgr().invalidateDb(extCatalog.getId(), getFullName()); } public boolean isInitialized() { @@ -163,7 +163,7 @@ private void buildMetaCache() { if (LOG.isDebugEnabled()) { LOG.debug("buildMetaCache for database: {}:{}", this.name, this.id, new Exception()); } - metaCache = Env.getCurrentEnv().getExtMetaCacheMgr().buildMetaCache( + metaCache = Env.getCurrentEnv().getExtMetaCacheMgr().legacyMetaCacheFactory().build( name, OptionalLong.of(Config.external_cache_expire_time_seconds_after_access), OptionalLong.of(Config.external_cache_refresh_time_minutes * 60L), 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 9b44833f01a67c..007e850e54e24e 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 @@ -17,40 +17,39 @@ package org.apache.doris.datasource; -import org.apache.doris.catalog.Type; -import org.apache.doris.cluster.ClusterNamespace; +import org.apache.doris.catalog.Env; import org.apache.doris.common.Config; -import org.apache.doris.common.Pair; import org.apache.doris.common.ThreadPoolManager; -import org.apache.doris.datasource.doris.DorisExternalMetaCacheMgr; -import org.apache.doris.datasource.hive.HMSExternalCatalog; -import org.apache.doris.datasource.hive.HMSExternalTable; -import org.apache.doris.datasource.hive.HiveMetaStoreCache; -import org.apache.doris.datasource.hudi.source.HudiCachedFsViewProcessor; -import org.apache.doris.datasource.hudi.source.HudiCachedMetaClientProcessor; -import org.apache.doris.datasource.hudi.source.HudiMetadataCacheMgr; -import org.apache.doris.datasource.hudi.source.HudiPartitionProcessor; -import org.apache.doris.datasource.iceberg.IcebergMetadataCache; -import org.apache.doris.datasource.maxcompute.MaxComputeMetadataCache; -import org.apache.doris.datasource.maxcompute.MaxComputeMetadataCacheMgr; -import org.apache.doris.datasource.metacache.MetaCache; -import org.apache.doris.datasource.mvcc.MvccUtil; -import org.apache.doris.datasource.paimon.PaimonMetadataCache; +import org.apache.doris.datasource.doris.DorisExternalMetaCache; +import org.apache.doris.datasource.hive.HiveExternalMetaCache; +import org.apache.doris.datasource.hudi.HudiExternalMetaCache; +import org.apache.doris.datasource.iceberg.IcebergExternalMetaCache; +import org.apache.doris.datasource.maxcompute.MaxComputeExternalMetaCache; +import org.apache.doris.datasource.metacache.AbstractExternalMetaCache; +import org.apache.doris.datasource.metacache.ExternalMetaCache; +import org.apache.doris.datasource.metacache.ExternalMetaCacheRegistry; +import org.apache.doris.datasource.metacache.ExternalMetaCacheRouteResolver; +import org.apache.doris.datasource.metacache.LegacyMetaCacheFactory; +import org.apache.doris.datasource.metacache.MetaCacheEntryDef; +import org.apache.doris.datasource.metacache.MetaCacheEntryInvalidation; +import org.apache.doris.datasource.metacache.MetaCacheEntryStats; +import org.apache.doris.datasource.paimon.PaimonExternalMetaCache; import org.apache.doris.fs.FileSystemCache; -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; +import java.util.ArrayList; +import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; -import java.util.OptionalLong; import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; +import javax.annotation.Nullable; /** * Cache meta of external catalog @@ -60,6 +59,14 @@ */ public class ExternalMetaCacheMgr { private static final Logger LOG = LogManager.getLogger(ExternalMetaCacheMgr.class); + private static final String ENTRY_SCHEMA = "schema"; + private static final String ENGINE_DEFAULT = "default"; + private static final String ENGINE_HIVE = "hive"; + private static final String ENGINE_HUDI = "hudi"; + private static final String ENGINE_ICEBERG = "iceberg"; + private static final String ENGINE_PAIMON = "paimon"; + private static final String ENGINE_MAXCOMPUTE = "maxcompute"; + private static final String ENGINE_DORIS = "doris"; /** * Executors for loading caches @@ -85,19 +92,14 @@ public class ExternalMetaCacheMgr { private ExecutorService fileListingExecutor; // This executor is used to schedule the getting split tasks private ExecutorService scheduleExecutor; + private final ExternalMetaCacheRegistry cacheRegistry; + private final ExternalMetaCacheRouteResolver routeResolver; + private final LegacyMetaCacheFactory legacyMetaCacheFactory; - private final CatalogScopedCacheMgr hiveMetaStoreCacheMgr; - private final CatalogScopedCacheMgr icebergMetadataCacheMgr; - private final CatalogScopedCacheMgr paimonMetadataCacheMgr; - private final CatalogScopedCacheMgr schemaCacheMgr; - // hudi partition manager - private final HudiMetadataCacheMgr hudiMetadataCacheMgr; // all catalogs could share the same fsCache. private FileSystemCache fsCache; // all external table row count cache. private ExternalRowCountCache rowCountCache; - private final MaxComputeMetadataCacheMgr maxComputeMetadataCacheMgr; - private final DorisExternalMetaCacheMgr dorisExternalMetaCacheMgr; public ExternalMetaCacheMgr(boolean isCheckpointCatalog) { rowCountRefreshExecutor = newThreadPool(isCheckpointCatalog, @@ -124,19 +126,11 @@ public ExternalMetaCacheMgr(boolean isCheckpointCatalog) { fsCache = new FileSystemCache(); rowCountCache = new ExternalRowCountCache(rowCountRefreshExecutor); + cacheRegistry = new ExternalMetaCacheRegistry(); + routeResolver = new ExternalMetaCacheRouteResolver(cacheRegistry); + legacyMetaCacheFactory = new LegacyMetaCacheFactory(commonRefreshExecutor); - hudiMetadataCacheMgr = new HudiMetadataCacheMgr(commonRefreshExecutor); - maxComputeMetadataCacheMgr = new MaxComputeMetadataCacheMgr(); - hiveMetaStoreCacheMgr = new CatalogScopedCacheMgr<>( - catalog -> new HiveMetaStoreCache((HMSExternalCatalog) catalog, - commonRefreshExecutor, fileListingExecutor)); - icebergMetadataCacheMgr = new CatalogScopedCacheMgr<>( - catalog -> new IcebergMetadataCache(catalog, commonRefreshExecutor)); - schemaCacheMgr = new CatalogScopedCacheMgr<>( - catalog -> new ExternalSchemaCache(catalog, commonRefreshExecutor)); - paimonMetadataCacheMgr = new CatalogScopedCacheMgr<>( - catalog -> new PaimonMetadataCache(catalog, commonRefreshExecutor)); - dorisExternalMetaCacheMgr = new DorisExternalMetaCacheMgr(commonRefreshExecutor); + initEngineCaches(); } private ExecutorService newThreadPool(boolean isCheckpointCatalog, int numThread, int queueSize, @@ -162,200 +156,248 @@ public ExecutorService getScheduleExecutor() { return scheduleExecutor; } - public HiveMetaStoreCache getMetaStoreCache(HMSExternalCatalog catalog) { - return hiveMetaStoreCacheMgr.getCache(catalog); + ExternalMetaCache engine(String engine) { + return cacheRegistry.resolve(engine); } - public ExternalSchemaCache getSchemaCache(ExternalCatalog catalog) { - return schemaCacheMgr.getCache(catalog); + public HiveExternalMetaCache hive(long catalogId) { + prepareCatalogByEngine(catalogId, ENGINE_HIVE); + return (HiveExternalMetaCache) engine(ENGINE_HIVE); } - public HudiPartitionProcessor getHudiPartitionProcess(ExternalCatalog catalog) { - return hudiMetadataCacheMgr.getPartitionProcessor(catalog); + public HudiExternalMetaCache hudi(long catalogId) { + prepareCatalogByEngine(catalogId, ENGINE_HUDI); + return (HudiExternalMetaCache) engine(ENGINE_HUDI); } - public HudiCachedFsViewProcessor getFsViewProcessor(ExternalCatalog catalog) { - return hudiMetadataCacheMgr.getFsViewProcessor(catalog); + public IcebergExternalMetaCache iceberg(long catalogId) { + prepareCatalogByEngine(catalogId, ENGINE_ICEBERG); + return (IcebergExternalMetaCache) engine(ENGINE_ICEBERG); } - public HudiCachedMetaClientProcessor getMetaClientProcessor(ExternalCatalog catalog) { - return hudiMetadataCacheMgr.getHudiMetaClientProcessor(catalog); + public PaimonExternalMetaCache paimon(long catalogId) { + prepareCatalogByEngine(catalogId, ENGINE_PAIMON); + return (PaimonExternalMetaCache) engine(ENGINE_PAIMON); } - public HudiMetadataCacheMgr getHudiMetadataCacheMgr() { - return hudiMetadataCacheMgr; + public MaxComputeExternalMetaCache maxCompute(long catalogId) { + prepareCatalogByEngine(catalogId, ENGINE_MAXCOMPUTE); + return (MaxComputeExternalMetaCache) engine(ENGINE_MAXCOMPUTE); } - public IcebergMetadataCache getIcebergMetadataCache(ExternalCatalog catalog) { - return icebergMetadataCacheMgr.getCache(catalog); + public DorisExternalMetaCache doris(long catalogId) { + prepareCatalogByEngine(catalogId, ENGINE_DORIS); + return (DorisExternalMetaCache) engine(ENGINE_DORIS); } - public PaimonMetadataCache getPaimonMetadataCache(ExternalCatalog catalog) { - return paimonMetadataCacheMgr.getCache(catalog); + public void prepareCatalog(long catalogId) { + Map catalogProperties = findCatalogProperties(catalogId); + if (catalogProperties == null) { + logMissingCatalogSkip(catalogId, "prepareCatalog"); + return; + } + routeCatalogEngines(catalogId, cache -> cache.initCatalog(catalogId, catalogProperties)); } - public MaxComputeMetadataCache getMaxComputeMetadataCache(long catalogId) { - return maxComputeMetadataCacheMgr.getMaxComputeMetadataCache(catalogId); + public void prepareCatalogByEngine(long catalogId, String engine) { + Map catalogProperties = findCatalogProperties(catalogId); + if (catalogProperties == null) { + logMissingCatalogSkip(catalogId, "prepareCatalogByEngine"); + return; + } + prepareCatalogByEngine(catalogId, engine, catalogProperties); } - public FileSystemCache getFsCache() { - return fsCache; + public void prepareCatalogByEngine(long catalogId, String engine, Map catalogProperties) { + Map safeCatalogProperties = catalogProperties == null + ? Maps.newHashMap() + : Maps.newHashMap(catalogProperties); + routeSpecifiedEngine(engine, cache -> cache.initCatalog(catalogId, safeCatalogProperties)); } - public ExternalRowCountCache getRowCountCache() { - return rowCountCache; + public void invalidateCatalog(long catalogId) { + routeCatalogEngines(catalogId, cache -> safeInvalidate( + cache, catalogId, "invalidateCatalog", + () -> cache.invalidateCatalogEntries(catalogId))); } - public DorisExternalMetaCacheMgr getDorisExternalMetaCacheMgr() { - return dorisExternalMetaCacheMgr; + public void invalidateCatalogByEngine(long catalogId, String engine) { + routeSpecifiedEngine(engine, cache -> safeInvalidate( + cache, catalogId, "invalidateCatalogByEngine", + () -> cache.invalidateCatalogEntries(catalogId))); } - public void removeCache(long catalogId) { - if (hiveMetaStoreCacheMgr.removeCache(catalogId) != null) { - LOG.info("remove hive metastore cache for catalog {}", catalogId); - } - if (schemaCacheMgr.removeCache(catalogId) != null) { - LOG.info("remove schema cache for catalog {}", catalogId); - } - if (icebergMetadataCacheMgr.removeCache(catalogId) != null) { - LOG.info("remove iceberg meta cache for catalog {}", catalogId); - } - hudiMetadataCacheMgr.removeCache(catalogId); - maxComputeMetadataCacheMgr.removeCache(catalogId); - PaimonMetadataCache paimonMetadataCache = paimonMetadataCacheMgr.removeCache(catalogId); - if (paimonMetadataCache != null) { - paimonMetadataCache.invalidateCatalogCache(catalogId); - } - dorisExternalMetaCacheMgr.removeCache(catalogId); + public void removeCatalog(long catalogId) { + routeCatalogEngines(catalogId, cache -> safeInvalidate( + cache, catalogId, "removeCatalog", + () -> cache.invalidateCatalog(catalogId))); } - public void invalidateTableCache(ExternalTable dorisTable) { - ExternalSchemaCache schemaCache = schemaCacheMgr.getCache(dorisTable.getCatalog().getId()); - if (schemaCache != null) { - schemaCache.invalidateTableCache(dorisTable); - } - HiveMetaStoreCache hiveMetaCache = hiveMetaStoreCacheMgr.getCache(dorisTable.getCatalog().getId()); - if (hiveMetaCache != null) { - hiveMetaCache.invalidateTableCache(dorisTable.getOrBuildNameMapping()); - } - IcebergMetadataCache icebergMetadataCache = icebergMetadataCacheMgr.getCache(dorisTable.getCatalog().getId()); - if (icebergMetadataCache != null) { - icebergMetadataCache.invalidateTableCache(dorisTable); - } - hudiMetadataCacheMgr.invalidateTableCache(dorisTable); - maxComputeMetadataCacheMgr.invalidateTableCache(dorisTable); - PaimonMetadataCache paimonMetadataCache = paimonMetadataCacheMgr.getCache(dorisTable.getCatalog().getId()); - if (paimonMetadataCache != null) { - paimonMetadataCache.invalidateTableCache(dorisTable); - } - if (LOG.isDebugEnabled()) { - LOG.debug("invalid table cache for {}.{} in catalog {}", dorisTable.getRemoteDbName(), - dorisTable.getRemoteName(), dorisTable.getCatalog().getName()); - } + public void removeCatalogByEngine(long catalogId, String engine) { + routeSpecifiedEngine(engine, cache -> safeInvalidate( + cache, catalogId, "removeCatalogByEngine", + () -> cache.invalidateCatalog(catalogId))); } - public void invalidateDbCache(long catalogId, String dbName) { - dbName = ClusterNamespace.getNameFromFullName(dbName); - ExternalSchemaCache schemaCache = schemaCacheMgr.getCache(catalogId); - if (schemaCache != null) { - schemaCache.invalidateDbCache(dbName); - } - HiveMetaStoreCache metaCache = hiveMetaStoreCacheMgr.getCache(catalogId); - if (metaCache != null) { - metaCache.invalidateDbCache(dbName); - } - IcebergMetadataCache icebergMetadataCache = icebergMetadataCacheMgr.getCache(catalogId); - if (icebergMetadataCache != null) { - icebergMetadataCache.invalidateDbCache(catalogId, dbName); - } - hudiMetadataCacheMgr.invalidateDbCache(catalogId, dbName); - maxComputeMetadataCacheMgr.invalidateDbCache(catalogId, dbName); - PaimonMetadataCache paimonMetadataCache = paimonMetadataCacheMgr.getCache(catalogId); - if (paimonMetadataCache != null) { - paimonMetadataCache.invalidateDbCache(catalogId, dbName); - } - if (LOG.isDebugEnabled()) { - LOG.debug("invalid db cache for {} in catalog {}", dbName, catalogId); - } + public void invalidateDb(long catalogId, String dbName) { + routeCatalogEngines(catalogId, cache -> safeInvalidate( + cache, catalogId, "invalidateDb", () -> cache.invalidateDb(catalogId, dbName))); + } + + public void invalidateTable(long catalogId, String dbName, String tableName) { + routeCatalogEngines(catalogId, cache -> safeInvalidate( + cache, catalogId, "invalidateTable", + () -> cache.invalidateTable(catalogId, dbName, tableName))); + } + + public void invalidateTableByEngine(long catalogId, String engine, String dbName, String tableName) { + routeSpecifiedEngine(engine, cache -> safeInvalidate( + cache, catalogId, "invalidateTableByEngine", + () -> cache.invalidateTable(catalogId, dbName, tableName))); + } + + public void invalidatePartitions(long catalogId, + String dbName, String tableName, List partitions) { + routeCatalogEngines(catalogId, cache -> safeInvalidate( + cache, catalogId, "invalidatePartitions", + () -> cache.invalidatePartitions(catalogId, dbName, tableName, partitions))); } - public void invalidateCatalogCache(long catalogId) { - schemaCacheMgr.removeCache(catalogId); - HiveMetaStoreCache metaCache = hiveMetaStoreCacheMgr.getCache(catalogId); - if (metaCache != null) { - metaCache.invalidateAll(); + public List getCatalogCacheStats(long catalogId) { + List stats = new ArrayList<>(); + cacheRegistry.allCaches().forEach(externalMetaCache -> externalMetaCache.stats(catalogId) + .forEach((entryName, entryStats) -> stats.add( + new CatalogMetaCacheStats(externalMetaCache.engine(), entryName, entryStats)))); + stats.sort(Comparator.comparing(CatalogMetaCacheStats::getEngineName) + .thenComparing(CatalogMetaCacheStats::getEntryName)); + return stats; + } + + public static final class CatalogMetaCacheStats { + private final String engineName; + private final String entryName; + private final MetaCacheEntryStats entryStats; + + public CatalogMetaCacheStats(String engineName, String entryName, MetaCacheEntryStats entryStats) { + this.engineName = Objects.requireNonNull(engineName, "engineName"); + this.entryName = Objects.requireNonNull(entryName, "entryName"); + this.entryStats = Objects.requireNonNull(entryStats, "entryStats"); } - IcebergMetadataCache icebergMetadataCache = icebergMetadataCacheMgr.getCache(catalogId); - if (icebergMetadataCache != null) { - icebergMetadataCache.invalidateCatalogCache(catalogId); + + public String getEngineName() { + return engineName; } - hudiMetadataCacheMgr.invalidateCatalogCache(catalogId); - maxComputeMetadataCacheMgr.invalidateCatalogCache(catalogId); - PaimonMetadataCache paimonMetadataCache = paimonMetadataCacheMgr.getCache(catalogId); - if (paimonMetadataCache != null) { - paimonMetadataCache.invalidateCatalogCache(catalogId); + + public String getEntryName() { + return entryName; } - dorisExternalMetaCacheMgr.invalidateCatalogCache(catalogId); - if (LOG.isDebugEnabled()) { - LOG.debug("invalid catalog cache for {}", catalogId); + + public MetaCacheEntryStats getEntryStats() { + return entryStats; } } - public void invalidSchemaCache(long catalogId) { - schemaCacheMgr.removeCache(catalogId); + private void initEngineCaches() { + registerBuiltinEngineCaches(); + } + + private void registerBuiltinEngineCaches() { + cacheRegistry.register(new DefaultExternalMetaCache(ENGINE_DEFAULT, commonRefreshExecutor)); + cacheRegistry.register(new HiveExternalMetaCache(commonRefreshExecutor, fileListingExecutor)); + cacheRegistry.register(new HudiExternalMetaCache(commonRefreshExecutor)); + cacheRegistry.register(new IcebergExternalMetaCache(commonRefreshExecutor)); + cacheRegistry.register(new PaimonExternalMetaCache(commonRefreshExecutor)); + cacheRegistry.register(new MaxComputeExternalMetaCache(commonRefreshExecutor)); + cacheRegistry.register(new DorisExternalMetaCache(commonRefreshExecutor)); + } + + private void routeCatalogEngines(long catalogId, Consumer action) { + routeResolver.resolveCatalogCaches(catalogId, getCatalog(catalogId)).forEach(action); + } + + private void routeSpecifiedEngine(String engine, Consumer action) { + action.accept(this.engine(engine)); } - public void addPartitionsCache(long catalogId, HMSExternalTable table, List partitionNames) { - String dbName = ClusterNamespace.getNameFromFullName(table.getDbName()); - HiveMetaStoreCache metaCache = hiveMetaStoreCacheMgr.getCache(catalogId); - if (metaCache != null) { - List partitionColumnTypes; - try { - partitionColumnTypes = table.getPartitionColumnTypes(MvccUtil.getSnapshotFromContext(table)); - } catch (NotSupportedException e) { - LOG.warn("Ignore not supported hms table, message: {} ", e.getMessage()); - return; + List resolveCatalogEngineNamesForTest(@Nullable CatalogIf catalog, long catalogId) { + List resolved = new ArrayList<>(); + routeResolver.resolveCatalogCaches(catalogId, catalog).forEach(cache -> resolved.add(cache.engine())); + return new ArrayList<>(resolved); + } + + private void safeInvalidate(ExternalMetaCache cache, long catalogId, String operation, Runnable action) { + if (!cache.isCatalogInitialized(catalogId)) { + if (LOG.isDebugEnabled()) { + LOG.debug("skip {} for catalog {} on engine '{}' because cache entry is absent", + operation, catalogId, cache.engine()); } - metaCache.addPartitionsCache(table.getOrBuildNameMapping(), partitionNames, partitionColumnTypes); - } - if (LOG.isDebugEnabled()) { - LOG.debug("add partition cache for {}.{} in catalog {}", dbName, table.getName(), catalogId); + return; } + action.run(); } - public void dropPartitionsCache(long catalogId, HMSExternalTable table, List partitionNames) { - String dbName = ClusterNamespace.getNameFromFullName(table.getDbName()); - HiveMetaStoreCache metaCache = hiveMetaStoreCacheMgr.getCache(catalogId); - if (metaCache != null) { - metaCache.dropPartitionsCache(table, partitionNames, true); + @Nullable + private Map findCatalogProperties(long catalogId) { + CatalogIf catalog = getCatalog(catalogId); + if (catalog == null) { + return null; + } + if (catalog.getProperties() == null) { + return Maps.newHashMap(); } + return Maps.newHashMap(catalog.getProperties()); + } + + private void logMissingCatalogSkip(long catalogId, String operation) { if (LOG.isDebugEnabled()) { - LOG.debug("drop partition cache for {}.{} in catalog {}", dbName, table.getName(), catalogId); + LOG.debug("skip {} for catalog {} because catalog does not exist", operation, catalogId); + } + } + + @Nullable + private CatalogIf getCatalog(long catalogId) { + if (Env.getCurrentEnv() == null || Env.getCurrentEnv().getCatalogMgr() == null) { + return null; } + return Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogId); } - public void invalidatePartitionsCache(ExternalTable dorisTable, List partitionNames) { - HiveMetaStoreCache metaCache = hiveMetaStoreCacheMgr.getCache(dorisTable.getCatalog().getId()); - if (metaCache != null) { - for (String partitionName : partitionNames) { - metaCache.invalidatePartitionCache(dorisTable, partitionName); + @SuppressWarnings("unchecked") + public Optional getSchemaCacheValue(ExternalTable table, SchemaCacheKey key) { + long catalogId = table.getCatalog().getId(); + String resolvedEngine = table.getMetaCacheEngine(); + prepareCatalogByEngine(catalogId, resolvedEngine); + try { + return ((ExternalMetaCache) engine(resolvedEngine)).getSchemaValue(catalogId, key); + } catch (IllegalStateException e) { + if (getCatalog(catalogId) != null) { + throw e; } + logMissingCatalogSkip(catalogId, "getSchemaCacheValue"); + return Optional.empty(); } + } + + public FileSystemCache getFsCache() { + return fsCache; + } + + public ExternalRowCountCache getRowCountCache() { + return rowCountCache; + } + + public void invalidateTableCache(ExternalTable dorisTable) { + invalidateTable(dorisTable.getCatalog().getId(), + dorisTable.getDbName(), + dorisTable.getName()); if (LOG.isDebugEnabled()) { - LOG.debug("invalidate partition cache for {}.{} in catalog {}", - dorisTable.getDbName(), dorisTable.getName(), dorisTable.getCatalog().getName()); + LOG.debug("invalid table cache for {}.{} in catalog {}", dorisTable.getRemoteDbName(), + dorisTable.getRemoteName(), dorisTable.getCatalog().getName()); } } - public MetaCache buildMetaCache(String name, - OptionalLong expireAfterAccessSec, OptionalLong refreshAfterWriteSec, long maxSize, - CacheLoader>> namesCacheLoader, - CacheLoader> metaObjCacheLoader, - RemovalListener> removalListener) { - MetaCache metaCache = new MetaCache<>( - name, commonRefreshExecutor, expireAfterAccessSec, refreshAfterWriteSec, - maxSize, namesCacheLoader, metaObjCacheLoader, removalListener); - return metaCache; + public LegacyMetaCacheFactory legacyMetaCacheFactory() { + return legacyMetaCacheFactory; } public static Map getCacheStats(CacheStats cacheStats, long estimatedSize) { @@ -368,4 +410,54 @@ public static Map getCacheStats(CacheStats cacheStats, long esti stats.put("estimated_size", String.valueOf(estimatedSize)); return stats; } + + void replaceEngineCachesForTest(List caches) { + cacheRegistry.resetForTest(caches); + } + + /** + * Fallback implementation of {@link AbstractExternalMetaCache} for engines that do not + * provide dedicated cache entries. + * + *

Registered entries: + *

    + *
  • {@code schema}: schema-only cache keyed by {@link SchemaCacheKey}
  • + *
+ * + *

This class keeps compatibility for generic external engines and routes only schema + * loading/invalidation. No engine-specific metadata (partitions/files/snapshots) is cached. + */ + private static class DefaultExternalMetaCache extends AbstractExternalMetaCache { + DefaultExternalMetaCache(String engine, ExecutorService refreshExecutor) { + super(engine, refreshExecutor); + registerEntry(MetaCacheEntryDef.of( + ENTRY_SCHEMA, + SchemaCacheKey.class, + SchemaCacheValue.class, + this::loadSchemaCacheValue, + defaultSchemaCacheSpec(), + MetaCacheEntryInvalidation.forTableIdentity( + key -> key.getNameMapping().getLocalDbName(), + key -> key.getNameMapping().getLocalTblName()))); + } + + @Override + protected Map catalogPropertyCompatibilityMap() { + return singleCompatibilityMap(ExternalCatalog.SCHEMA_CACHE_TTL_SECOND, ENTRY_SCHEMA); + } + + private SchemaCacheValue loadSchemaCacheValue(SchemaCacheKey key) { + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(key.getNameMapping().getCtlId()); + if (!(catalog instanceof ExternalCatalog)) { + throw new CacheException("catalog %s is not external when loading schema cache", + null, key.getNameMapping().getCtlId()); + } + ExternalCatalog externalCatalog = (ExternalCatalog) catalog; + return externalCatalog.getSchema(key).orElseThrow(() -> new CacheException( + "failed to load schema cache value for: %s.%s.%s", + null, key.getNameMapping().getCtlId(), + key.getNameMapping().getLocalDbName(), + key.getNameMapping().getLocalTblName())); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalSchemaCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalSchemaCache.java deleted file mode 100644 index cf129ea8623f6c..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalSchemaCache.java +++ /dev/null @@ -1,151 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.datasource; - -import org.apache.doris.common.CacheFactory; -import org.apache.doris.common.Config; -import org.apache.doris.datasource.metacache.CacheSpec; -import org.apache.doris.metric.GaugeMetric; -import org.apache.doris.metric.Metric; -import org.apache.doris.metric.MetricLabel; -import org.apache.doris.metric.MetricRepo; - -import com.github.benmanes.caffeine.cache.LoadingCache; -import lombok.Data; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.Optional; -import java.util.OptionalLong; -import java.util.Set; -import java.util.concurrent.ExecutorService; - -// The schema cache for external table -public class ExternalSchemaCache { - private static final Logger LOG = LogManager.getLogger(ExternalSchemaCache.class); - private final ExternalCatalog catalog; - - private LoadingCache> schemaCache; - - public ExternalSchemaCache(ExternalCatalog catalog, ExecutorService executor) { - this.catalog = catalog; - init(executor); - initMetrics(); - } - - private void init(ExecutorService executor) { - CacheSpec cacheSpec = CacheSpec.fromTtlValue( - catalog.getProperties().get(ExternalCatalog.SCHEMA_CACHE_TTL_SECOND), - Config.external_cache_expire_time_seconds_after_access, - Config.max_external_schema_cache_num); - CacheFactory schemaCacheFactory = new CacheFactory( - CacheSpec.toExpireAfterAccess(cacheSpec.getTtlSecond()), - OptionalLong.of(Config.external_cache_refresh_time_minutes * 60), - cacheSpec.getCapacity(), - false, - null); - schemaCache = schemaCacheFactory.buildCache(this::loadSchema, executor); - } - - private void initMetrics() { - // schema cache - GaugeMetric schemaCacheGauge = new GaugeMetric("external_schema_cache", - Metric.MetricUnit.NOUNIT, "external schema cache number") { - @Override - public Long getValue() { - return schemaCache.estimatedSize(); - } - }; - schemaCacheGauge.addLabel(new MetricLabel("catalog", catalog.getName())); - MetricRepo.DORIS_METRIC_REGISTER.addMetrics(schemaCacheGauge); - } - - private Optional loadSchema(SchemaCacheKey key) { - Optional schema = catalog.getSchema(key); - if (schema.isPresent()) { - schema.get().validateSchema(); - } - if (LOG.isDebugEnabled()) { - LOG.debug("load schema for {} in catalog {}", key, catalog.getName()); - } - return schema; - } - - public Optional getSchemaValue(SchemaCacheKey key) { - return schemaCache.get(key); - } - - public void invalidateTableCache(ExternalTable dorisTable) { - schemaCache.asMap().keySet().stream() - .filter(key -> key.getNameMapping().getLocalDbName().equals(dorisTable.getDbName()) - && key.getNameMapping().getLocalTblName().equals(dorisTable.getName())) - .forEach(schemaCache::invalidate); - } - - public void invalidateDbCache(String dbName) { - long start = System.currentTimeMillis(); - Set keys = schemaCache.asMap().keySet(); - for (SchemaCacheKey key : keys) { - if (key.getNameMapping().getLocalDbName().equals(dbName)) { - schemaCache.invalidate(key); - } - } - if (LOG.isDebugEnabled()) { - LOG.debug("invalid schema cache for db {} in catalog {} cost: {} ms", dbName, catalog.getName(), - (System.currentTimeMillis() - start)); - } - } - - public void invalidateAll() { - schemaCache.invalidateAll(); - if (LOG.isDebugEnabled()) { - LOG.debug("invalid all schema cache in catalog {}", catalog.getName()); - } - } - - @Data - public static class SchemaCacheKey { - private NameMapping nameMapping; - - public SchemaCacheKey(NameMapping nameMapping) { - this.nameMapping = nameMapping; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (!(obj instanceof SchemaCacheKey)) { - return false; - } - return nameMapping.equals(((SchemaCacheKey) obj).nameMapping); - } - - @Override - public int hashCode() { - return nameMapping.hashCode(); - } - - @Override - public String toString() { - return "SchemaCacheKey{" + "dbName='" - + nameMapping.getLocalDbName() + '\'' + ", tblName='" + nameMapping.getLocalTblName() + '\'' + '}'; - } - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java index 4a81bb3c7f7193..691f91bb08ddf0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java @@ -31,7 +31,6 @@ import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.common.util.Util; -import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey; import org.apache.doris.datasource.mvcc.MvccSnapshot; import org.apache.doris.nereids.rules.expression.rules.SortedPartitionRanges; import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; @@ -174,8 +173,7 @@ public TableType getType() { @Override public List getFullSchema() { - ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); - Optional schemaCacheValue = cache.getSchemaValue(new SchemaCacheKey(getOrBuildNameMapping())); + Optional schemaCacheValue = getSchemaCacheValue(); return schemaCacheValue.map(SchemaCacheValue::getSchema).orElse(null); } @@ -214,6 +212,13 @@ public String getEngine() { return getType().toEngineName(); } + /** + * Returns the effective meta cache engine for this table. + */ + public String getMetaCacheEngine() { + return "default"; + } + @Override public String getMysqlType() { return getType().toMysqlType(); @@ -396,8 +401,8 @@ public List getChunkSizes() { } public Optional getSchemaCacheValue() { - ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); - return cache.getSchemaValue(new SchemaCacheKey(getOrBuildNameMapping())); + return Env.getCurrentEnv().getExtMetaCacheMgr() + .getSchemaCacheValue(this, new SchemaCacheKey(getOrBuildNameMapping())); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/NameMapping.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/NameMapping.java index 192b86d0995987..60c8fc1a3d4bb6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/NameMapping.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/NameMapping.java @@ -46,6 +46,11 @@ public static NameMapping createForTest(String dbName, String tblName) { return new NameMapping(0, dbName, tblName, dbName, tblName); } + @VisibleForTesting + public static NameMapping createForTest(long ctlId, String dbName, String tblName) { + return new NameMapping(ctlId, dbName, tblName, dbName, tblName); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/cache/ManifestCacheKey.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/SchemaCacheKey.java similarity index 58% rename from fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/cache/ManifestCacheKey.java rename to fe/fe-core/src/main/java/org/apache/doris/datasource/SchemaCacheKey.java index 41b52187aec3f5..4810319b2def1e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/cache/ManifestCacheKey.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/SchemaCacheKey.java @@ -15,44 +15,37 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.datasource.iceberg.cache; +package org.apache.doris.datasource; -import java.util.Objects; +import lombok.Data; -/** - * Cache key for a single Iceberg manifest file. - * Since Iceberg manifest files are immutable, path uniquely identifies a manifest. - */ -public class ManifestCacheKey { - private final String path; +@Data +public class SchemaCacheKey { + private final NameMapping nameMapping; - public ManifestCacheKey(String path) { - this.path = path; - } - - public String getPath() { - return path; + public SchemaCacheKey(NameMapping nameMapping) { + this.nameMapping = nameMapping; } @Override - public boolean equals(Object o) { - if (this == o) { + public boolean equals(Object obj) { + if (this == obj) { return true; } - if (!(o instanceof ManifestCacheKey)) { + if (!(obj instanceof SchemaCacheKey)) { return false; } - ManifestCacheKey that = (ManifestCacheKey) o; - return Objects.equals(path, that.path); + return nameMapping.equals(((SchemaCacheKey) obj).nameMapping); } @Override public int hashCode() { - return Objects.hash(path); + return nameMapping.hashCode(); } @Override public String toString() { - return "ManifestCacheKey{path='" + path + "'}"; + return "SchemaCacheKey{" + "dbName='" + + nameMapping.getLocalDbName() + '\'' + ", tblName='" + nameMapping.getLocalTblName() + '\'' + '}'; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/SchemaCacheValue.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/SchemaCacheValue.java index 4611a0d9816fbe..ac7c55105af2ca 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/SchemaCacheValue.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/SchemaCacheValue.java @@ -24,7 +24,7 @@ import java.util.Set; /** - * The cache value of ExternalSchemaCache. + * The cache value of schema cache entries. * Different external table type has different schema cache value. * For example, Hive table has HMSSchemaCacheValue, Paimon table has PaimonSchemaCacheValue. * All objects that should be refreshed along with schema should be put in this class. diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/doris/DorisExternalMetaCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/doris/DorisExternalMetaCache.java new file mode 100644 index 00000000000000..d14ba5645bf269 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/doris/DorisExternalMetaCache.java @@ -0,0 +1,133 @@ +// 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.datasource.doris; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.Config; +import org.apache.doris.datasource.CacheException; +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.SchemaCacheKey; +import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.doris.datasource.metacache.AbstractExternalMetaCache; +import org.apache.doris.datasource.metacache.CacheSpec; +import org.apache.doris.datasource.metacache.MetaCacheEntry; +import org.apache.doris.datasource.metacache.MetaCacheEntryDef; +import org.apache.doris.datasource.metacache.MetaCacheEntryInvalidation; +import org.apache.doris.system.Backend; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.stream.Collectors; + +/** + * Remote Doris engine implementation of {@link AbstractExternalMetaCache}. + * + *

Registered entries: + *

    + *
  • {@code backends}: remote backend topology keyed by catalog id
  • + *
  • {@code schema}: schema cache keyed by {@link SchemaCacheKey}
  • + *
+ * + *

The backend cache is intentionally independent from table/db invalidation and can be + * refreshed explicitly via {@link #invalidateBackendCache(long)}. + * + *

db/table/partition invalidation only targets schema entries. + */ +public class DorisExternalMetaCache extends AbstractExternalMetaCache { + private static final Logger LOG = LogManager.getLogger(DorisExternalMetaCache.class); + + public static final String ENGINE = "doris"; + public static final String ENTRY_BACKENDS = "backends"; + public static final String ENTRY_SCHEMA = "schema"; + @SuppressWarnings("unchecked") + private static final Class> BACKEND_MAP_CLASS = + (Class>) (Class) ImmutableMap.class; + private static final String BACKEND_ENTRY_KEY = "backends"; + + private final EntryHandle> backendsEntry; + private final EntryHandle schemaEntry; + + public DorisExternalMetaCache(ExecutorService refreshExecutor) { + super(ENGINE, refreshExecutor); + backendsEntry = registerEntry(MetaCacheEntryDef.contextualOnly( + ENTRY_BACKENDS, + String.class, + BACKEND_MAP_CLASS, + CacheSpec.of(true, Config.external_cache_expire_time_seconds_after_access, 20))); + schemaEntry = registerEntry(MetaCacheEntryDef.of( + ENTRY_SCHEMA, + SchemaCacheKey.class, + SchemaCacheValue.class, + this::loadSchemaCacheValue, + defaultSchemaCacheSpec(), + MetaCacheEntryInvalidation.forNameMapping(SchemaCacheKey::getNameMapping))); + } + + @Override + public Collection aliases() { + return Collections.singleton("external_doris"); + } + + public ImmutableMap getBackends(long catalogId) { + ImmutableMap backends = backendsEntry.get(catalogId) + .get(BACKEND_ENTRY_KEY, ignored -> loadBackends(catalogId)); + return backends == null ? ImmutableMap.of() : backends; + } + + public void invalidateBackendCache(long catalogId) { + MetaCacheEntry> backends = backendsEntry.getIfInitialized(catalogId); + if (backends != null) { + backends.invalidateKey(BACKEND_ENTRY_KEY); + } + } + + private ImmutableMap loadBackends(long catalogId) { + RemoteDorisExternalCatalog catalog = (RemoteDorisExternalCatalog) Env.getCurrentEnv().getCatalogMgr() + .getCatalog(catalogId); + List backends = catalog.getFeServiceClient().listBackends(); + if (LOG.isDebugEnabled()) { + List names = backends.stream().map(Backend::getAddress).collect(Collectors.toList()); + LOG.debug("load backends:{} from:{}", String.join(",", names), catalog.getName()); + } + Map backendMap = Maps.newHashMap(); + backends.forEach(backend -> backendMap.put(backend.getId(), backend)); + return ImmutableMap.copyOf(backendMap); + } + + private SchemaCacheValue loadSchemaCacheValue(SchemaCacheKey key) { + ExternalTable dorisTable = findExternalTable(key.getNameMapping(), ENGINE); + return dorisTable.initSchemaAndUpdateTime(key).orElseThrow(() -> + new CacheException("failed to load doris schema cache value for: %s.%s.%s", + null, key.getNameMapping().getCtlId(), key.getNameMapping().getLocalDbName(), + key.getNameMapping().getLocalTblName())); + } + + @Override + protected Map catalogPropertyCompatibilityMap() { + return singleCompatibilityMap(ExternalCatalog.SCHEMA_CACHE_TTL_SECOND, ENTRY_SCHEMA); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/doris/DorisExternalMetaCacheMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/doris/DorisExternalMetaCacheMgr.java deleted file mode 100644 index 70f92853ccc526..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/doris/DorisExternalMetaCacheMgr.java +++ /dev/null @@ -1,83 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.datasource.doris; - -import org.apache.doris.catalog.Env; -import org.apache.doris.common.CacheFactory; -import org.apache.doris.common.Config; -import org.apache.doris.system.Backend; - -import com.github.benmanes.caffeine.cache.LoadingCache; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Maps; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.List; -import java.util.Map; -import java.util.OptionalLong; -import java.util.concurrent.ExecutorService; -import java.util.stream.Collectors; - -public class DorisExternalMetaCacheMgr { - private static final Logger LOG = LogManager.getLogger(DorisExternalMetaCacheMgr.class); - private final LoadingCache> backendsCache; - - public DorisExternalMetaCacheMgr(ExecutorService executor) { - CacheFactory cacheFactory = new CacheFactory( - OptionalLong.of(Config.external_cache_expire_time_seconds_after_access), - OptionalLong.of(Config.external_cache_refresh_time_minutes * 60), - 20, - true, - null); - backendsCache = cacheFactory.buildCache(key -> loadBackends(key), executor); - } - - private ImmutableMap loadBackends(Long catalogId) { - RemoteDorisExternalCatalog catalog = (RemoteDorisExternalCatalog) Env.getCurrentEnv().getCatalogMgr() - .getCatalog(catalogId); - List backends = catalog.getFeServiceClient().listBackends(); - if (LOG.isDebugEnabled()) { - List names = backends.stream().map(b -> b.getAddress()).collect(Collectors.toList()); - LOG.debug("load backends:{} from:{}", String.join(",", names), catalog.getName()); - } - Map backendMap = Maps.newHashMap(); - backends.forEach(backend -> backendMap.put(backend.getId(), backend)); - return ImmutableMap.copyOf(backendMap); - } - - public void removeCache(long catalogId) { - backendsCache.invalidate(catalogId); - } - - public void invalidateBackendCache(long catalogId) { - backendsCache.invalidate(catalogId); - } - - public void invalidateCatalogCache(long catalogId) { - invalidateBackendCache(catalogId); - } - - public ImmutableMap getBackends(long catalogId) { - ImmutableMap backends = backendsCache.get(catalogId); - if (backends == null) { - return ImmutableMap.of(); - } - return backends; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/doris/RemoteDorisExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/doris/RemoteDorisExternalTable.java index c86d91af92ba48..a4d3f92e77da0a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/doris/RemoteDorisExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/doris/RemoteDorisExternalTable.java @@ -52,6 +52,11 @@ public RemoteDorisExternalTable(long id, String name, String remoteName, super(id, name, remoteName, catalog, db, TableType.DORIS_EXTERNAL_TABLE); } + @Override + public String getMetaCacheEngine() { + return DorisExternalMetaCache.ENGINE; + } + @Override protected synchronized void makeSureInitialized() { super.makeSureInitialized(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/doris/RemoteOlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/doris/RemoteOlapTable.java index 80089b3a14cdce..d17e8dd015f56e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/doris/RemoteOlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/doris/RemoteOlapTable.java @@ -105,12 +105,15 @@ public void rebuildPartitions(List oldPartitions, List upd public void invalidateBackendsIfNeed() { ImmutableMap backends = - Env.getCurrentEnv().getExtMetaCacheMgr().getDorisExternalMetaCacheMgr().getBackends(catalog.getId()); + Env.getCurrentEnv().getExtMetaCacheMgr() + .doris(catalog.getId()) + .getBackends(catalog.getId()); for (Partition partition : getPartitions()) { for (Tablet tablet : partition.getBaseIndex().getTablets()) { for (long backendId : tablet.getBackendIds()) { if (!backends.containsKey(backendId)) { - Env.getCurrentEnv().getExtMetaCacheMgr().getDorisExternalMetaCacheMgr() + Env.getCurrentEnv().getExtMetaCacheMgr() + .doris(catalog.getId()) .invalidateBackendCache(catalog.getId()); return; } @@ -125,6 +128,8 @@ public long getCatalogId() { } public ImmutableMap getAllBackendsByAllCluster() { - return Env.getCurrentEnv().getExtMetaCacheMgr().getDorisExternalMetaCacheMgr().getBackends(catalog.getId()); + return Env.getCurrentEnv().getExtMetaCacheMgr() + .doris(catalog.getId()) + .getBackends(catalog.getId()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/AcidUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/AcidUtil.java index 93dee780605ee3..8c961808dca990 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/AcidUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/AcidUtil.java @@ -20,7 +20,7 @@ import org.apache.doris.backup.Status; import org.apache.doris.common.util.LocationPath; import org.apache.doris.datasource.hive.AcidInfo.DeleteDeltaInfo; -import org.apache.doris.datasource.hive.HiveMetaStoreCache.FileCacheValue; +import org.apache.doris.datasource.hive.HiveExternalMetaCache.FileCacheValue; import org.apache.doris.datasource.property.storage.StorageProperties; import org.apache.doris.fs.FileSystem; import org.apache.doris.fs.remote.RemoteFile; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java index fe86390b581f93..e2d73fd7a16edf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java @@ -28,8 +28,10 @@ import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.InitCatalogLog; import org.apache.doris.datasource.SessionContext; +import org.apache.doris.datasource.hudi.HudiExternalMetaCache; import org.apache.doris.datasource.iceberg.IcebergMetadataOps; import org.apache.doris.datasource.iceberg.IcebergUtils; +import org.apache.doris.datasource.metacache.CacheSpec; import org.apache.doris.datasource.operations.ExternalMetadataOperations; import org.apache.doris.datasource.property.metastore.AbstractHiveProperties; import org.apache.doris.fs.FileSystemProvider; @@ -214,7 +216,11 @@ public void notifyPropertiesUpdated(Map updatedProps) { String fileMetaCacheTtl = updatedProps.getOrDefault(FILE_META_CACHE_TTL_SECOND, null); String partitionCacheTtl = updatedProps.getOrDefault(PARTITION_CACHE_TTL_SECOND, null); if (Objects.nonNull(fileMetaCacheTtl) || Objects.nonNull(partitionCacheTtl)) { - Env.getCurrentEnv().getExtMetaCacheMgr().getMetaStoreCache(this).init(); + Env.getCurrentEnv().getExtMetaCacheMgr().removeCatalogByEngine(getId(), HiveExternalMetaCache.ENGINE); + } + if (updatedProps.keySet().stream() + .anyMatch(key -> CacheSpec.isMetaCacheKeyForEngine(key, HudiExternalMetaCache.ENGINE))) { + Env.getCurrentEnv().getExtMetaCacheMgr().removeCatalogByEngine(getId(), HudiExternalMetaCache.ENGINE); } } @@ -236,4 +242,3 @@ public IcebergMetadataOps getIcebergMetadataOps() { return icebergMetadataOps; } } - diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java index 6bb5c2019b3ee6..137ee718741e99 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java @@ -32,16 +32,18 @@ import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.UserException; -import org.apache.doris.datasource.ExternalSchemaCache; -import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey; import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.SchemaCacheKey; import org.apache.doris.datasource.SchemaCacheValue; import org.apache.doris.datasource.TablePartitionValues; +import org.apache.doris.datasource.hudi.HudiExternalMetaCache; import org.apache.doris.datasource.hudi.HudiSchemaCacheKey; import org.apache.doris.datasource.hudi.HudiSchemaCacheValue; import org.apache.doris.datasource.hudi.HudiUtils; +import org.apache.doris.datasource.iceberg.IcebergExternalMetaCache; import org.apache.doris.datasource.iceberg.IcebergMvccSnapshot; import org.apache.doris.datasource.iceberg.IcebergSchemaCacheKey; +import org.apache.doris.datasource.iceberg.IcebergSnapshotCacheValue; import org.apache.doris.datasource.iceberg.IcebergUtils; import org.apache.doris.datasource.mvcc.EmptyMvccSnapshot; import org.apache.doris.datasource.mvcc.MvccSnapshot; @@ -208,6 +210,23 @@ public HMSExternalTable(long id, String name, String remoteName, HMSExternalCata super(id, name, remoteName, catalog, db, TableType.HMS_EXTERNAL_TABLE); } + @Override + public String getMetaCacheEngine() { + switch (getDlaType()) { + case HIVE: + return HiveExternalMetaCache.ENGINE; + case HUDI: + return HudiExternalMetaCache.ENGINE; + case ICEBERG: + return IcebergExternalMetaCache.ENGINE; + case UNKNOWN: + default: + throw new IllegalArgumentException( + String.format("unsupported HMS DLA type '%s' for table %s.%s.%s in catalog %d", + getDlaType(), getCatalog().getName(), getDbName(), getName(), getCatalog().getId())); + } + } + // Will throw NotSupportedException if not supported hms table. // Otherwise, return true. public boolean isSupportedHmsTable() { @@ -343,17 +362,30 @@ public org.apache.hadoop.hive.metastore.api.Table getRemoteTable() { @Override public List getFullSchema() { makeSureInitialized(); - ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); if (getDlaType() == DLAType.HUDI) { return ((HudiDlaTable) dlaTable).getHudiSchemaCacheValue(MvccUtil.getSnapshotFromContext(this)) .getSchema(); } else if (getDlaType() == DLAType.ICEBERG) { return IcebergUtils.getIcebergSchema(this); } - Optional schemaCacheValue = cache.getSchemaValue(new SchemaCacheKey(getOrBuildNameMapping())); + Optional schemaCacheValue = getSchemaCacheValue(); return schemaCacheValue.map(SchemaCacheValue::getSchema).orElse(null); } + @Override + public Optional getSchemaCacheValue() { + makeSureInitialized(); + if (dlaType == DLAType.HUDI) { + return Optional.of( + ((HudiDlaTable) dlaTable).getHudiSchemaCacheValue(MvccUtil.getSnapshotFromContext(this))); + } else if (dlaType == DLAType.ICEBERG) { + IcebergSnapshotCacheValue snapshotValue = IcebergUtils.getSnapshotCacheValue( + MvccUtil.getSnapshotFromContext(this), this); + return Optional.of(IcebergUtils.getSchemaCacheValue(this, snapshotValue)); + } + return super.getSchemaCacheValue(); + } + public List getPartitionColumnTypes(Optional snapshot) { makeSureInitialized(); if (getDlaType() == DLAType.HUDI) { @@ -366,9 +398,8 @@ public List getPartitionColumnTypes(Optional snapshot) { public List getHudiPartitionColumnTypes(long timestamp) { makeSureInitialized(); - ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); - Optional schemaCacheValue = cache.getSchemaValue( - new HudiSchemaCacheKey(getOrBuildNameMapping(), timestamp)); + Optional schemaCacheValue = Env.getCurrentEnv().getExtMetaCacheMgr() + .getSchemaCacheValue(this, new HudiSchemaCacheKey(getOrBuildNameMapping(), timestamp)); return schemaCacheValue.map(value -> ((HMSSchemaCacheValue) value).getPartitionColTypes()) .orElse(Collections.emptyList()); } @@ -396,7 +427,7 @@ public Optional> getSortedPartitionRanges(CatalogR if (CollectionUtils.isEmpty(this.getPartitionColumns())) { return Optional.empty(); } - HiveMetaStoreCache.HivePartitionValues hivePartitionValues = getHivePartitionValues( + HiveExternalMetaCache.HivePartitionValues hivePartitionValues = getHivePartitionValues( MvccUtil.getSnapshotFromContext(this)); return hivePartitionValues.getSortedPartitionRanges(); } @@ -431,7 +462,7 @@ public Map getNameToPartitionItems() { if (CollectionUtils.isEmpty(this.getPartitionColumns())) { return Collections.emptyMap(); } - HiveMetaStoreCache.HivePartitionValues hivePartitionValues = getHivePartitionValues( + HiveExternalMetaCache.HivePartitionValues hivePartitionValues = getHivePartitionValues( MvccUtil.getSnapshotFromContext(this)); Map idToPartitionItem = hivePartitionValues.getIdToPartitionItem(); // transfer id to name @@ -814,9 +845,7 @@ public Optional getColumnStatistic(String colName) { case ICEBERG: if (GlobalVariable.enableFetchIcebergStats) { return StatisticsUtil.getIcebergColumnStats(colName, - Env.getCurrentEnv().getExtMetaCacheMgr() - .getIcebergMetadataCache(this.getCatalog()) - .getIcebergTable(this)); + IcebergUtils.getIcebergTable(this)); } else { break; } @@ -907,11 +936,11 @@ public void gsonPostProcess() throws IOException { @Override public List getChunkSizes() { - HiveMetaStoreCache.HivePartitionValues partitionValues = getAllPartitionValues(); - List filesByPartitions = getFilesForPartitions(partitionValues, 0); + HiveExternalMetaCache.HivePartitionValues partitionValues = getAllPartitionValues(); + List filesByPartitions = getFilesForPartitions(partitionValues, 0); List result = Lists.newArrayList(); - for (HiveMetaStoreCache.FileCacheValue files : filesByPartitions) { - for (HiveMetaStoreCache.HiveFileStatus file : files.getFiles()) { + for (HiveExternalMetaCache.FileCacheValue files : filesByPartitions) { + for (HiveExternalMetaCache.HiveFileStatus file : files.getFiles()) { result.add(file.getLength()); } } @@ -991,10 +1020,10 @@ public MTMVSnapshotIf getTableSnapshot(Optional snapshot) throws A @Override public long getNewestUpdateVersionOrTime() { - HiveMetaStoreCache.HivePartitionValues hivePartitionValues = getHivePartitionValues( + HiveExternalMetaCache.HivePartitionValues hivePartitionValues = getHivePartitionValues( MvccUtil.getSnapshotFromContext(this)); - HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getMetaStoreCache((HMSExternalCatalog) getCatalog()); + HiveExternalMetaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() + .hive(getCatalog().getId()); List partitionList = cache.getAllPartitionsWithCache(this, Lists.newArrayList(hivePartitionValues.getPartitionValuesMap().values())); if (CollectionUtils.isEmpty(partitionList)) { @@ -1022,16 +1051,16 @@ private long getRowCountFromFileList() { } long rows = UNKNOWN_ROW_COUNT; try { - HiveMetaStoreCache.HivePartitionValues partitionValues = getAllPartitionValues(); + HiveExternalMetaCache.HivePartitionValues partitionValues = getAllPartitionValues(); // Get files for all partitions. int samplePartitionSize = Config.hive_stats_partition_sample_size; - List filesByPartitions = + List filesByPartitions = getFilesForPartitions(partitionValues, samplePartitionSize); LOG.info("Number of files selected for hive table {} is {}", name, filesByPartitions.size()); long totalSize = 0; // Calculate the total file size. - for (HiveMetaStoreCache.FileCacheValue files : filesByPartitions) { - for (HiveMetaStoreCache.HiveFileStatus file : files.getFiles()) { + for (HiveExternalMetaCache.FileCacheValue files : filesByPartitions) { + for (HiveExternalMetaCache.HiveFileStatus file : files.getFiles()) { totalSize += file.getLength(); } } @@ -1066,13 +1095,13 @@ private long getRowCountFromFileList() { } // Get all partition values from cache. - private HiveMetaStoreCache.HivePartitionValues getAllPartitionValues() { + private HiveExternalMetaCache.HivePartitionValues getAllPartitionValues() { if (isView()) { return null; } Optional snapshot = MvccUtil.getSnapshotFromContext(this); List partitionColumnTypes = getPartitionColumnTypes(snapshot); - HiveMetaStoreCache.HivePartitionValues partitionValues = null; + HiveExternalMetaCache.HivePartitionValues partitionValues = null; // Get table partitions from cache. if (!partitionColumnTypes.isEmpty()) { // It is ok to get partition values from cache, @@ -1091,13 +1120,13 @@ private HiveMetaStoreCache.HivePartitionValues getAllPartitionValues() { // Get all files related to given partition values // If sampleSize > 0, randomly choose part of partitions of the whole table. - private List getFilesForPartitions( - HiveMetaStoreCache.HivePartitionValues partitionValues, int sampleSize) { + private List getFilesForPartitions( + HiveExternalMetaCache.HivePartitionValues partitionValues, int sampleSize) { if (isView()) { return Lists.newArrayList(); } - HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getMetaStoreCache((HMSExternalCatalog) catalog); + HiveExternalMetaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() + .hive(getCatalog().getId()); List hivePartitions = Lists.newArrayList(); if (partitionValues != null) { Map idToPartitionItem = partitionValues.getIdToPartitionItem(); @@ -1168,11 +1197,8 @@ public boolean firstColumnIsString() { public HoodieTableMetaClient getHudiClient() { return Env.getCurrentEnv() .getExtMetaCacheMgr() - .getMetaClientProcessor(getCatalog()) - .getHoodieTableMetaClient( - getOrBuildNameMapping(), - getRemoteTable().getSd().getLocation(), - getCatalog().getConfiguration()); + .hudi(getCatalog().getId()) + .getHoodieTableMetaClient(getOrBuildNameMapping()); } public boolean isValidRelatedTable() { @@ -1238,19 +1264,19 @@ private Table loadHiveTable() { return client.getTable(getRemoteDbName(), remoteName); } - public HiveMetaStoreCache.HivePartitionValues getHivePartitionValues(Optional snapshot) { - HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getMetaStoreCache((HMSExternalCatalog) this.getCatalog()); + public HiveExternalMetaCache.HivePartitionValues getHivePartitionValues(Optional snapshot) { + HiveExternalMetaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() + .hive(getCatalog().getId()); try { List partitionColumnTypes = this.getPartitionColumnTypes(snapshot); return cache.getPartitionValues(this, partitionColumnTypes); } catch (Exception e) { - if (e.getMessage().contains(HiveMetaStoreCache.ERR_CACHE_INCONSISTENCY)) { + if (e.getMessage().contains(HiveExternalMetaCache.ERR_CACHE_INCONSISTENCY)) { LOG.warn("Hive metastore cache inconsistency detected for table: {}.{}.{}. " + "Clearing cache and retrying to get partition values.", this.getCatalog().getName(), this.getDbName(), this.getName(), e); - ExternalSchemaCache schemaCache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); - schemaCache.invalidateTableCache(this); + Env.getCurrentEnv().getExtMetaCacheMgr().invalidateTableByEngine( + getCatalog().getId(), getMetaCacheEngine(), getDbName(), getName()); List partitionColumnTypes = this.getPartitionColumnTypes(snapshot); return cache.getPartitionValues(this, partitionColumnTypes); } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveDlaTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveDlaTable.java index c49081de60da31..bfd39c59684e65 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveDlaTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveDlaTable.java @@ -71,10 +71,10 @@ public Map getAndCopyPartitionItems(Optional snapshot) throws AnalysisException { - HiveMetaStoreCache.HivePartitionValues hivePartitionValues = hmsTable.getHivePartitionValues(snapshot); + HiveExternalMetaCache.HivePartitionValues hivePartitionValues = hmsTable.getHivePartitionValues(snapshot); Long partitionId = getPartitionIdByNameOrAnalysisException(partitionName, hivePartitionValues); - HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getMetaStoreCache((HMSExternalCatalog) hmsTable.getCatalog()); + HiveExternalMetaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() + .hive(hmsTable.getCatalog().getId()); HivePartition hivePartition = getHivePartitionByIdOrAnalysisException(partitionId, hivePartitionValues, cache); return new MTMVTimestampSnapshot(hivePartition.getLastModifiedTime()); @@ -94,9 +94,9 @@ public MTMVSnapshotIf getTableSnapshot(Optional snapshot) throws A HivePartition maxPartition = null; long maxVersionTime = 0L; long visibleVersionTime; - HiveMetaStoreCache.HivePartitionValues hivePartitionValues = hmsTable.getHivePartitionValues(snapshot); - HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getMetaStoreCache((HMSExternalCatalog) hmsTable.getCatalog()); + HiveExternalMetaCache.HivePartitionValues hivePartitionValues = hmsTable.getHivePartitionValues(snapshot); + HiveExternalMetaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() + .hive(hmsTable.getCatalog().getId()); List partitionList = cache.getAllPartitionsWithCache(hmsTable, Lists.newArrayList(hivePartitionValues.getPartitionValuesMap().values())); if (CollectionUtils.isEmpty(partitionList)) { @@ -114,7 +114,7 @@ public MTMVSnapshotIf getTableSnapshot(Optional snapshot) throws A } private Long getPartitionIdByNameOrAnalysisException(String partitionName, - HiveMetaStoreCache.HivePartitionValues hivePartitionValues) + HiveExternalMetaCache.HivePartitionValues hivePartitionValues) throws AnalysisException { Long partitionId = hivePartitionValues.getPartitionNameToIdMap().get(partitionName); if (partitionId == null) { @@ -124,8 +124,8 @@ private Long getPartitionIdByNameOrAnalysisException(String partitionName, } private HivePartition getHivePartitionByIdOrAnalysisException(Long partitionId, - HiveMetaStoreCache.HivePartitionValues hivePartitionValues, - HiveMetaStoreCache cache) throws AnalysisException { + HiveExternalMetaCache.HivePartitionValues hivePartitionValues, + HiveExternalMetaCache cache) throws AnalysisException { List partitionValues = hivePartitionValues.getPartitionValuesMap().get(partitionId); if (CollectionUtils.isEmpty(partitionValues)) { throw new AnalysisException("can not find partitionValues: " + partitionId); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveExternalMetaCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveExternalMetaCache.java new file mode 100644 index 00000000000000..2ba8fe2459bef3 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveExternalMetaCache.java @@ -0,0 +1,1095 @@ +// 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.datasource.hive; + +import org.apache.doris.analysis.PartitionValue; +import org.apache.doris.backup.Status.ErrCode; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ListPartitionItem; +import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.catalog.PartitionKey; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.Type; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; +import org.apache.doris.common.FeConstants; +import org.apache.doris.common.UserException; +import org.apache.doris.common.security.authentication.AuthenticationConfig; +import org.apache.doris.common.security.authentication.HadoopAuthenticator; +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.CatalogIf; +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.SchemaCacheKey; +import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.doris.datasource.metacache.AbstractExternalMetaCache; +import org.apache.doris.datasource.metacache.CacheSpec; +import org.apache.doris.datasource.metacache.MetaCacheEntry; +import org.apache.doris.datasource.metacache.MetaCacheEntryDef; +import org.apache.doris.fs.DirectoryLister; +import org.apache.doris.fs.FileSystemCache; +import org.apache.doris.fs.FileSystemDirectoryLister; +import org.apache.doris.fs.FileSystemIOException; +import org.apache.doris.fs.RemoteIterator; +import org.apache.doris.fs.remote.RemoteFile; +import org.apache.doris.fs.remote.RemoteFileSystem; +import org.apache.doris.nereids.rules.expression.rules.SortedPartitionRanges; +import org.apache.doris.planner.ListPartitionPrunerV2; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.google.common.collect.BiMap; +import com.google.common.collect.HashBiMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Streams; +import lombok.Data; +import org.apache.hadoop.fs.BlockLocation; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.utils.FileUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.stream.Collectors; + +/** + * Hive engine implementation of {@link AbstractExternalMetaCache}. + * + *

This cache consolidates schema metadata and Hive Metastore-derived runtime metadata + * under one engine so callers can use a unified invalidation path. + * + *

Registered entries: + *

    + *
  • {@code schema}: table schema cache keyed by {@link SchemaCacheKey}
  • + *
  • {@code partition_values}: partition value/index structures per table
  • + *
  • {@code partition}: single partition metadata keyed by partition values
  • + *
  • {@code file}: file listing cache for partition/table locations
  • + *
+ * + *

Invalidation behavior: + *

    + *
  • {@link #invalidateDb(long, String)} and {@link #invalidateTable(long, String, String)} + * clear all related entries with table/db granularity.
  • + *
  • {@link #invalidatePartitions(long, String, String, List)} supports partition-level + * invalidation when specific partition names are provided, and falls back to table-level + * invalidation for empty input or unresolved table metadata.
  • + *
+ */ +public class HiveExternalMetaCache extends AbstractExternalMetaCache { + private static final Logger LOG = LogManager.getLogger(HiveExternalMetaCache.class); + + public static final String ENGINE = "hive"; + public static final String ENTRY_SCHEMA = "schema"; + public static final String ENTRY_PARTITION_VALUES = "partition_values"; + public static final String ENTRY_PARTITION = "partition"; + public static final String ENTRY_FILE = "file"; + + public static final String HIVE_DEFAULT_PARTITION = "__HIVE_DEFAULT_PARTITION__"; + public static final String ERR_CACHE_INCONSISTENCY = "ERR_CACHE_INCONSISTENCY: "; + + private final ExecutorService fileListingExecutor; + + private final EntryHandle schemaEntry; + private final EntryHandle partitionValuesEntry; + private final EntryHandle partitionEntry; + private final EntryHandle fileEntry; + private final PartitionCacheCoordinator partitionCacheCoordinator = new PartitionCacheCoordinator(); + + public HiveExternalMetaCache(ExecutorService refreshExecutor, ExecutorService fileListingExecutor) { + super(ENGINE, refreshExecutor); + this.fileListingExecutor = fileListingExecutor; + + schemaEntry = registerEntry(MetaCacheEntryDef.of( + ENTRY_SCHEMA, + SchemaCacheKey.class, + SchemaCacheValue.class, + this::loadSchemaCacheValue, + defaultSchemaCacheSpec())); + partitionValuesEntry = registerEntry(MetaCacheEntryDef.of( + ENTRY_PARTITION_VALUES, + PartitionValueCacheKey.class, + HivePartitionValues.class, + this::loadPartitionValuesCacheValue, + CacheSpec.of( + true, + Config.external_cache_expire_time_seconds_after_access, + Config.max_hive_partition_table_cache_num))); + partitionEntry = registerEntry(MetaCacheEntryDef.of( + ENTRY_PARTITION, + PartitionCacheKey.class, + HivePartition.class, + this::loadPartitionCacheValue, + CacheSpec.of( + true, + Config.external_cache_expire_time_seconds_after_access, + Config.max_hive_partition_cache_num))); + fileEntry = registerEntry(MetaCacheEntryDef.of( + ENTRY_FILE, + FileCacheKey.class, + FileCacheValue.class, + this::loadFileCacheValue, + CacheSpec.of( + true, + Config.external_cache_expire_time_seconds_after_access, + Config.max_external_file_cache_num))); + } + + @Override + public Collection aliases() { + return Collections.singleton("hms"); + } + + public void refreshCatalog(long catalogId) { + invalidateCatalog(catalogId); + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogId); + Map catalogProperties = catalog == null || catalog.getProperties() == null + ? Maps.newHashMap() + : Maps.newHashMap(catalog.getProperties()); + initCatalog(catalogId, catalogProperties); + } + + @Override + public void invalidateDb(long catalogId, String dbName) { + schemaEntry.get(catalogId).invalidateIf(key -> matchDb(key.getNameMapping(), dbName)); + partitionValuesEntry.get(catalogId).invalidateIf(key -> matchDb(key.getNameMapping(), dbName)); + partitionEntry.get(catalogId).invalidateIf(key -> matchDb(key.getNameMapping(), dbName)); + fileEntry.get(catalogId).invalidateAll(); + } + + @Override + public void invalidateTable(long catalogId, String dbName, String tableName) { + schemaEntry.get(catalogId).invalidateIf(key -> matchTable(key.getNameMapping(), dbName, tableName)); + partitionValuesEntry.get(catalogId).invalidateIf(key -> matchTable(key.getNameMapping(), dbName, tableName)); + partitionEntry.get(catalogId).invalidateIf(key -> matchTable(key.getNameMapping(), dbName, tableName)); + long tableId = Util.genIdByName(dbName, tableName); + fileEntry.get(catalogId).invalidateIf(key -> key.isSameTable(tableId)); + } + + @Override + public void invalidatePartitions(long catalogId, String dbName, String tableName, List partitions) { + if (partitions == null || partitions.isEmpty()) { + invalidateTable(catalogId, dbName, tableName); + return; + } + + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogId); + if (!(catalog instanceof HMSExternalCatalog)) { + return; + } + + HMSExternalCatalog hmsCatalog = (HMSExternalCatalog) catalog; + if (hmsCatalog.getDbNullable(dbName) == null + || !(hmsCatalog.getDbNullable(dbName).getTableNullable(tableName) instanceof HMSExternalTable)) { + invalidateTable(catalogId, dbName, tableName); + return; + } + HMSExternalTable hmsTable = (HMSExternalTable) hmsCatalog.getDbNullable(dbName).getTableNullable(tableName); + + for (String partition : partitions) { + invalidatePartitionCache(hmsTable, partition); + } + } + + @Override + protected Map catalogPropertyCompatibilityMap() { + Map compatibilityMap = Maps.newHashMap(); + compatibilityMap.put(ExternalCatalog.SCHEMA_CACHE_TTL_SECOND, metaCacheTtlKey(ENTRY_SCHEMA)); + compatibilityMap.put(HMSExternalCatalog.PARTITION_CACHE_TTL_SECOND, metaCacheTtlKey(ENTRY_PARTITION_VALUES)); + compatibilityMap.put(HMSExternalCatalog.FILE_META_CACHE_TTL_SECOND, metaCacheTtlKey(ENTRY_FILE)); + return compatibilityMap; + } + + private MetaCacheEntry schemaEntryIfInitialized(long catalogId) { + return schemaEntry.getIfInitialized(catalogId); + } + + private MetaCacheEntry partitionValuesEntryIfInitialized( + long catalogId) { + return partitionValuesEntry.getIfInitialized(catalogId); + } + + private MetaCacheEntry partitionEntryIfInitialized(long catalogId) { + return partitionEntry.getIfInitialized(catalogId); + } + + private MetaCacheEntry fileEntryIfInitialized(long catalogId) { + return fileEntry.getIfInitialized(catalogId); + } + + private SchemaCacheValue loadSchemaCacheValue(SchemaCacheKey key) { + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(key.getNameMapping().getCtlId()); + if (!(catalog instanceof ExternalCatalog)) { + throw new CacheException("catalog %s is not external when loading hive schema cache", + null, key.getNameMapping().getCtlId()); + } + ExternalCatalog externalCatalog = (ExternalCatalog) catalog; + return externalCatalog.getSchema(key).orElseThrow(() -> new CacheException( + "failed to load hive schema cache value for: %s.%s.%s", + null, key.getNameMapping().getCtlId(), + key.getNameMapping().getLocalDbName(), + key.getNameMapping().getLocalTblName())); + } + + private HivePartitionValues loadPartitionValuesCacheValue(PartitionValueCacheKey key) { + return loadPartitionValues(key); + } + + private HivePartition loadPartitionCacheValue(PartitionCacheKey key) { + return loadPartition(key); + } + + private FileCacheValue loadFileCacheValue(FileCacheKey key) { + return loadFiles(key, new FileSystemDirectoryLister(), null); + } + + private HMSExternalCatalog hmsCatalog(long catalogId) { + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogId); + if (!(catalog instanceof HMSExternalCatalog)) { + throw new CacheException("catalog %s is not hms when loading hive metastore cache", null, catalogId); + } + return (HMSExternalCatalog) catalog; + } + + private HivePartitionValues loadPartitionValues(PartitionValueCacheKey key) { + NameMapping nameMapping = key.nameMapping; + HMSExternalCatalog catalog = hmsCatalog(nameMapping.getCtlId()); + List partitionNames = catalog.getClient() + .listPartitionNames(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName()); + if (LOG.isDebugEnabled()) { + LOG.debug("load #{} partitions for {} in catalog {}", partitionNames.size(), key, catalog.getName()); + } + + Map idToPartitionItem = Maps.newHashMapWithExpectedSize(partitionNames.size()); + BiMap partitionNameToIdMap = HashBiMap.create(partitionNames.size()); + String localDbName = nameMapping.getLocalDbName(); + String localTblName = nameMapping.getLocalTblName(); + for (String partitionName : partitionNames) { + long partitionId = Util.genIdByName(catalog.getName(), localDbName, localTblName, partitionName); + ListPartitionItem listPartitionItem = toListPartitionItem(partitionName, key.types, catalog.getName()); + idToPartitionItem.put(partitionId, listPartitionItem); + partitionNameToIdMap.put(partitionName, partitionId); + } + + Map> partitionValuesMap = ListPartitionPrunerV2.getPartitionValuesMap(idToPartitionItem); + return new HivePartitionValues(idToPartitionItem, partitionNameToIdMap, partitionValuesMap); + } + + private ListPartitionItem toListPartitionItem(String partitionName, List types, String catalogName) { + List partitionValues = HiveUtil.toPartitionValues(partitionName); + Preconditions.checkState(types != null, + ERR_CACHE_INCONSISTENCY + "partition types is null for partition " + partitionName); + Preconditions.checkState(partitionValues.size() == types.size(), + ERR_CACHE_INCONSISTENCY + partitionName + " vs. " + types); + + List values = Lists.newArrayListWithExpectedSize(types.size()); + for (String partitionValue : partitionValues) { + values.add(new PartitionValue(partitionValue, HIVE_DEFAULT_PARTITION.equals(partitionValue))); + } + try { + PartitionKey partitionKey = PartitionKey.createListPartitionKeyWithTypes(values, types, true); + return new ListPartitionItem(Lists.newArrayList(partitionKey)); + } catch (AnalysisException e) { + throw new CacheException("failed to convert hive partition %s to list partition in catalog %s", + e, partitionName, catalogName); + } + } + + private HivePartition loadPartition(PartitionCacheKey key) { + NameMapping nameMapping = key.nameMapping; + HMSExternalCatalog catalog = hmsCatalog(nameMapping.getCtlId()); + Partition partition = catalog.getClient() + .getPartition(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName(), key.values); + StorageDescriptor sd = partition.getSd(); + if (LOG.isDebugEnabled()) { + LOG.debug("load partition format: {}, location: {} for {} in catalog {}", + sd.getInputFormat(), sd.getLocation(), key, catalog.getName()); + } + return new HivePartition(nameMapping, false, sd.getInputFormat(), sd.getLocation(), key.values, + partition.getParameters()); + } + + private Map loadPartitions(Iterable keys) { + Map result = new HashMap<>(); + if (keys == null) { + return result; + } + + List keyList = Streams.stream(keys).collect(Collectors.toList()); + if (keyList.isEmpty()) { + return result; + } + + PartitionCacheKey oneKey = keyList.get(0); + NameMapping nameMapping = oneKey.nameMapping; + HMSExternalCatalog catalog = hmsCatalog(nameMapping.getCtlId()); + + String localDbName = nameMapping.getLocalDbName(); + String localTblName = nameMapping.getLocalTblName(); + List partitionColumns = ((HMSExternalTable) catalog.getDbNullable(localDbName) + .getTableNullable(localTblName)).getPartitionColumns(); + + List partitionNames = keyList.stream().map(key -> { + StringBuilder sb = new StringBuilder(); + Preconditions.checkState(key.getValues().size() == partitionColumns.size()); + for (int i = 0; i < partitionColumns.size(); i++) { + sb.append(FileUtils.escapePathName(partitionColumns.get(i).getName())); + sb.append("="); + sb.append(FileUtils.escapePathName(key.getValues().get(i))); + sb.append("/"); + } + sb.delete(sb.length() - 1, sb.length()); + return sb.toString(); + }).collect(Collectors.toList()); + + List partitions = catalog.getClient().getPartitions( + nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName(), partitionNames); + for (Partition partition : partitions) { + StorageDescriptor sd = partition.getSd(); + result.put(new PartitionCacheKey(nameMapping, partition.getValues()), + new HivePartition(nameMapping, false, + sd.getInputFormat(), sd.getLocation(), partition.getValues(), + partition.getParameters())); + } + return result; + } + + private FileCacheValue getFileCache(HMSExternalCatalog catalog, + LocationPath path, + String inputFormat, + List partitionValues, + DirectoryLister directoryLister, + TableIf table) throws UserException { + FileCacheValue result = new FileCacheValue(); + + FileSystemCache.FileSystemCacheKey fileSystemCacheKey = new FileSystemCache.FileSystemCacheKey( + path.getFsIdentifier(), path.getStorageProperties()); + RemoteFileSystem fs = Env.getCurrentEnv().getExtMetaCacheMgr().getFsCache() + .getRemoteFileSystem(fileSystemCacheKey); + result.setSplittable(HiveUtil.isSplittable(fs, inputFormat, path.getNormalizedLocation())); + + boolean isRecursiveDirectories = Boolean.valueOf( + catalog.getProperties().getOrDefault("hive.recursive_directories", "true")); + try { + RemoteIterator iterator = directoryLister.listFiles(fs, isRecursiveDirectories, + table, path.getNormalizedLocation()); + while (iterator.hasNext()) { + RemoteFile remoteFile = iterator.next(); + String srcPath = remoteFile.getPath().toString(); + LocationPath locationPath = LocationPath.of(srcPath, path.getStorageProperties()); + result.addFile(remoteFile, locationPath); + } + } catch (FileSystemIOException e) { + if (e.getErrorCode().isPresent() && e.getErrorCode().get().equals(ErrCode.NOT_FOUND)) { + LOG.warn("File {} not exist.", path.getNormalizedLocation()); + if (!Boolean.valueOf(catalog.getProperties() + .getOrDefault("hive.ignore_absent_partitions", "true"))) { + throw new UserException("Partition location does not exist: " + path.getNormalizedLocation()); + } + } else { + throw new RuntimeException(e); + } + } + + result.setPartitionValues(Lists.newArrayList(partitionValues)); + return result; + } + + private FileCacheValue loadFiles(FileCacheKey key, DirectoryLister directoryLister, TableIf table) { + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + HMSExternalCatalog catalog = hmsCatalog(key.catalogId); + try { + Thread.currentThread().setContextClassLoader(ClassLoader.getSystemClassLoader()); + LocationPath finalLocation = LocationPath.of( + key.getLocation(), catalog.getCatalogProperty().getStoragePropertiesMap()); + try { + FileCacheValue result = getFileCache(catalog, finalLocation, key.inputFormat, + key.getPartitionValues(), directoryLister, table); + for (int i = 0; i < result.getValuesSize(); i++) { + if (HIVE_DEFAULT_PARTITION.equals(result.getPartitionValues().get(i))) { + result.getPartitionValues().set(i, FeConstants.null_string); + } + } + + if (LOG.isDebugEnabled()) { + LOG.debug("load #{} splits for {} in catalog {}", + result.getFiles().size(), key, catalog.getName()); + } + return result; + } catch (Exception e) { + throw new CacheException("failed to get input splits for %s in catalog %s", + e, key, catalog.getName()); + } + } finally { + Thread.currentThread().setContextClassLoader(classLoader); + } + } + + public HivePartitionValues getPartitionValues(ExternalTable dorisTable, List types) { + PartitionValueCacheKey key = new PartitionValueCacheKey(dorisTable.getOrBuildNameMapping(), types); + return getPartitionValues(key); + } + + @VisibleForTesting + public HivePartitionValues getPartitionValues(PartitionValueCacheKey key) { + return partitionValuesEntry.get(key.nameMapping.getCtlId()).get(key); + } + + public List getFilesByPartitions(List partitions, + boolean withCache, + boolean concurrent, + DirectoryLister directoryLister, + TableIf table) { + long start = System.currentTimeMillis(); + if (partitions.isEmpty()) { + return Lists.newArrayList(); + } + + HivePartition firstPartition = partitions.get(0); + long catalogId = firstPartition.getNameMapping().getCtlId(); + long fileId = Util.genIdByName(firstPartition.getNameMapping().getLocalDbName(), + firstPartition.getNameMapping().getLocalTblName()); + List keys = partitions.stream().map(p -> p.isDummyPartition() + ? FileCacheKey.createDummyCacheKey(catalogId, fileId, p.getPath(), p.getInputFormat()) + : new FileCacheKey(catalogId, fileId, p.getPath(), p.getInputFormat(), p.getPartitionValues())) + .collect(Collectors.toList()); + + List fileLists; + try { + if (withCache) { + MetaCacheEntry fileEntry = this.fileEntry.get(catalogId); + fileLists = keys.stream().map(fileEntry::get).collect(Collectors.toList()); + } else if (concurrent) { + List> futures = keys.stream().map( + key -> fileListingExecutor.submit(() -> loadFiles(key, directoryLister, table))) + .collect(Collectors.toList()); + fileLists = Lists.newArrayListWithExpectedSize(keys.size()); + for (Future future : futures) { + fileLists.add(future.get()); + } + } else { + fileLists = keys.stream() + .map(key -> loadFiles(key, directoryLister, table)) + .collect(Collectors.toList()); + } + } catch (ExecutionException e) { + throw new CacheException("failed to get files from partitions in catalog %s", + e, hmsCatalog(catalogId).getName()); + } catch (InterruptedException e) { + throw new CacheException("failed to get files from partitions in catalog %s with interrupted exception", + e, hmsCatalog(catalogId).getName()); + } + + if (LOG.isDebugEnabled()) { + LOG.debug("get #{} files from #{} partitions in catalog {} cost: {} ms", + fileLists.stream().mapToInt(l -> l.getFiles() == null ? 0 : l.getFiles().size()).sum(), + partitions.size(), hmsCatalog(catalogId).getName(), (System.currentTimeMillis() - start)); + } + return fileLists; + } + + public HivePartition getHivePartition(ExternalTable dorisTable, List partitionValues) { + NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); + return partitionEntry.get(nameMapping.getCtlId()).get(new PartitionCacheKey(nameMapping, partitionValues)); + } + + public List getAllPartitionsWithCache(ExternalTable dorisTable, + List> partitionValuesList) { + return getAllPartitions(dorisTable, partitionValuesList, true); + } + + public List getAllPartitionsWithoutCache(ExternalTable dorisTable, + List> partitionValuesList) { + return getAllPartitions(dorisTable, partitionValuesList, false); + } + + private List getAllPartitions(ExternalTable dorisTable, + List> partitionValuesList, + boolean withCache) { + long start = System.currentTimeMillis(); + NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); + long catalogId = nameMapping.getCtlId(); + List keys = partitionValuesList.stream() + .map(p -> new PartitionCacheKey(nameMapping, p)) + .collect(Collectors.toList()); + + List partitions; + if (withCache) { + MetaCacheEntry partitionEntry = this.partitionEntry.get(catalogId); + partitions = keys.stream().map(partitionEntry::get).collect(Collectors.toList()); + } else { + partitions = new ArrayList<>(loadPartitions(keys).values()); + } + + if (LOG.isDebugEnabled()) { + LOG.debug("get #{} partitions in catalog {} cost: {} ms", partitions.size(), + hmsCatalog(catalogId).getName(), (System.currentTimeMillis() - start)); + } + return partitions; + } + + public void invalidateTableCache(NameMapping nameMapping) { + long catalogId = nameMapping.getCtlId(); + + MetaCacheEntry partitionValuesEntry = + partitionValuesEntryIfInitialized(catalogId); + if (partitionValuesEntry != null) { + partitionValuesEntry.invalidateKey(new PartitionValueCacheKey(nameMapping, null)); + } + + MetaCacheEntry partitionEntry = partitionEntryIfInitialized(catalogId); + if (partitionEntry != null) { + partitionEntry.invalidateIf(k -> k.isSameTable( + nameMapping.getLocalDbName(), nameMapping.getLocalTblName())); + } + + MetaCacheEntry fileEntry = fileEntryIfInitialized(catalogId); + if (fileEntry != null) { + long tableId = Util.genIdByName(nameMapping.getLocalDbName(), nameMapping.getLocalTblName()); + fileEntry.invalidateIf(k -> k.isSameTable(tableId)); + } + } + + public void invalidatePartitionCache(ExternalTable dorisTable, String partitionName) { + partitionCacheCoordinator.invalidatePartitionCache(dorisTable, partitionName); + } + + /** + * Selectively refreshes cache for affected partitions based on update information from BE. + */ + public void refreshAffectedPartitions(HMSExternalTable table, + List partitionUpdates, + List modifiedPartNames, + List newPartNames) { + partitionCacheCoordinator.refreshAffectedPartitions(table, partitionUpdates, modifiedPartNames, newPartNames); + } + + public void refreshAffectedPartitionsCache(HMSExternalTable table, + List modifiedPartNames, + List newPartNames) { + partitionCacheCoordinator.refreshAffectedPartitionsCache(table, modifiedPartNames, newPartNames); + } + + public void addPartitionsCache(NameMapping nameMapping, + List partitionNames, + List partitionColumnTypes) { + partitionCacheCoordinator.addPartitionsCache(nameMapping, partitionNames, partitionColumnTypes); + } + + public void dropPartitionsCache(ExternalTable dorisTable, + List partitionNames, + boolean invalidPartitionCache) { + partitionCacheCoordinator.dropPartitionsCache(dorisTable, partitionNames, invalidPartitionCache); + } + + private final class PartitionCacheCoordinator { + private void invalidatePartitionCache(ExternalTable dorisTable, String partitionName) { + NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); + long catalogId = nameMapping.getCtlId(); + + MetaCacheEntry partitionValuesEntry = + partitionValuesEntryIfInitialized(catalogId); + MetaCacheEntry partitionEntry = partitionEntryIfInitialized(catalogId); + MetaCacheEntry fileEntry = fileEntryIfInitialized(catalogId); + if (partitionValuesEntry == null || partitionEntry == null || fileEntry == null) { + return; + } + + long tableId = Util.genIdByName(nameMapping.getLocalDbName(), nameMapping.getLocalTblName()); + PartitionValueCacheKey key = new PartitionValueCacheKey(nameMapping, null); + HivePartitionValues partitionValues = partitionValuesEntry.getIfPresent(key); + if (partitionValues == null) { + return; + } + + Long partitionId = partitionValues.partitionNameToIdMap.get(partitionName); + if (partitionId == null) { + return; + } + + List values = partitionValues.partitionValuesMap.get(partitionId); + if (values == null) { + return; + } + + PartitionCacheKey partKey = new PartitionCacheKey(nameMapping, values); + HivePartition partition = partitionEntry.getIfPresent(partKey); + if (partition == null) { + return; + } + + fileEntry.invalidateKey(new FileCacheKey(nameMapping.getCtlId(), tableId, partition.getPath(), + null, partition.getPartitionValues())); + partitionEntry.invalidateKey(partKey); + } + + private void refreshAffectedPartitions(HMSExternalTable table, + List partitionUpdates, + List modifiedPartNames, + List newPartNames) { + if (partitionUpdates == null || partitionUpdates.isEmpty()) { + return; + } + + for (org.apache.doris.thrift.THivePartitionUpdate update : partitionUpdates) { + String partitionName = update.getName(); + if (Strings.isNullOrEmpty(partitionName)) { + continue; + } + + switch (update.getUpdateMode()) { + case APPEND: + case OVERWRITE: + modifiedPartNames.add(partitionName); + break; + case NEW: + newPartNames.add(partitionName); + break; + default: + LOG.warn("Unknown update mode {} for partition {}", + update.getUpdateMode(), partitionName); + break; + } + } + + refreshAffectedPartitionsCache(table, modifiedPartNames, newPartNames); + } + + private void refreshAffectedPartitionsCache(HMSExternalTable table, + List modifiedPartNames, + List newPartNames) { + for (String partitionName : modifiedPartNames) { + invalidatePartitionCache(table, partitionName); + } + + List mergedPartNames = Lists.newArrayList(modifiedPartNames); + mergedPartNames.addAll(newPartNames); + if (!mergedPartNames.isEmpty()) { + addPartitionsCache(table.getOrBuildNameMapping(), mergedPartNames, + table.getPartitionColumnTypes(java.util.Optional.empty())); + } + + LOG.info("Refreshed cache for table {}: {} modified partitions, {} new partitions", + table.getName(), modifiedPartNames.size(), newPartNames.size()); + } + + private void addPartitionsCache(NameMapping nameMapping, + List partitionNames, + List partitionColumnTypes) { + long catalogId = nameMapping.getCtlId(); + MetaCacheEntry partitionValuesEntry = + partitionValuesEntryIfInitialized(catalogId); + if (partitionValuesEntry == null) { + return; + } + + PartitionValueCacheKey key = new PartitionValueCacheKey(nameMapping, partitionColumnTypes); + HivePartitionValues partitionValues = partitionValuesEntry.getIfPresent(key); + if (partitionValues == null) { + return; + } + + HivePartitionValues copy = partitionValues.copy(); + Map idToPartitionItemBefore = copy.getIdToPartitionItem(); + Map partitionNameToIdMapBefore = copy.getPartitionNameToIdMap(); + Map idToPartitionItem = new HashMap<>(); + + HMSExternalCatalog catalog = hmsCatalog(catalogId); + String localDbName = nameMapping.getLocalDbName(); + String localTblName = nameMapping.getLocalTblName(); + for (String partitionName : partitionNames) { + if (partitionNameToIdMapBefore.containsKey(partitionName)) { + LOG.info("addPartitionsCache partitionName:[{}] has exist in table:[{}]", + partitionName, localTblName); + continue; + } + long partitionId = Util.genIdByName(catalog.getName(), localDbName, localTblName, partitionName); + ListPartitionItem listPartitionItem = toListPartitionItem(partitionName, key.types, catalog.getName()); + idToPartitionItemBefore.put(partitionId, listPartitionItem); + idToPartitionItem.put(partitionId, listPartitionItem); + partitionNameToIdMapBefore.put(partitionName, partitionId); + } + + Map> partitionValuesMapBefore = copy.getPartitionValuesMap(); + Map> partitionValuesMap = ListPartitionPrunerV2.getPartitionValuesMap(idToPartitionItem); + partitionValuesMapBefore.putAll(partitionValuesMap); + copy.rebuildSortedPartitionRanges(); + + HivePartitionValues partitionValuesCur = partitionValuesEntry.getIfPresent(key); + if (partitionValuesCur == partitionValues) { + partitionValuesEntry.put(key, copy); + } + } + + private void dropPartitionsCache(ExternalTable dorisTable, + List partitionNames, + boolean invalidPartitionCache) { + NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); + long catalogId = nameMapping.getCtlId(); + + MetaCacheEntry partitionValuesEntry = + partitionValuesEntryIfInitialized(catalogId); + if (partitionValuesEntry == null) { + return; + } + + PartitionValueCacheKey key = new PartitionValueCacheKey(nameMapping, null); + HivePartitionValues partitionValues = partitionValuesEntry.getIfPresent(key); + if (partitionValues == null) { + return; + } + + HivePartitionValues copy = partitionValues.copy(); + Map partitionNameToIdMapBefore = copy.getPartitionNameToIdMap(); + Map idToPartitionItemBefore = copy.getIdToPartitionItem(); + Map> partitionValuesMap = copy.getPartitionValuesMap(); + + for (String partitionName : partitionNames) { + if (!partitionNameToIdMapBefore.containsKey(partitionName)) { + LOG.info("dropPartitionsCache partitionName:[{}] not exist in table:[{}]", + partitionName, nameMapping.getFullLocalName()); + continue; + } + Long partitionId = partitionNameToIdMapBefore.remove(partitionName); + idToPartitionItemBefore.remove(partitionId); + partitionValuesMap.remove(partitionId); + + if (invalidPartitionCache) { + invalidatePartitionCache(dorisTable, partitionName); + } + } + + copy.rebuildSortedPartitionRanges(); + HivePartitionValues partitionValuesCur = partitionValuesEntry.getIfPresent(key); + if (partitionValuesCur == partitionValues) { + partitionValuesEntry.put(key, copy); + } + } + } + + @VisibleForTesting + public void putPartitionValuesCacheForTest(PartitionValueCacheKey key, HivePartitionValues values) { + partitionValuesEntry.get(key.getNameMapping().getCtlId()).put(key, values); + } + + public List getFilesByTransaction(List partitions, + Map txnValidIds, + boolean isFullAcid, + String bindBrokerName) { + List fileCacheValues = Lists.newArrayList(); + try { + if (partitions.isEmpty()) { + return fileCacheValues; + } + for (HivePartition partition : partitions) { + HMSExternalCatalog catalog = hmsCatalog(partition.getNameMapping().getCtlId()); + LocationPath locationPath = LocationPath.of(partition.getPath(), + catalog.getCatalogProperty().getStoragePropertiesMap()); + RemoteFileSystem fileSystem = Env.getCurrentEnv().getExtMetaCacheMgr().getFsCache() + .getRemoteFileSystem(new FileSystemCache.FileSystemCacheKey( + locationPath.getNormalizedLocation(), + locationPath.getStorageProperties())); + AuthenticationConfig authenticationConfig = AuthenticationConfig + .getKerberosConfig(locationPath.getStorageProperties().getBackendConfigProperties()); + HadoopAuthenticator hadoopAuthenticator = + HadoopAuthenticator.getHadoopAuthenticator(authenticationConfig); + + fileCacheValues.add( + hadoopAuthenticator.doAs(() -> AcidUtil.getAcidState( + fileSystem, + partition, + txnValidIds, + catalog.getCatalogProperty().getStoragePropertiesMap(), + isFullAcid))); + } + } catch (Exception e) { + throw new CacheException("Failed to get input splits %s", e, txnValidIds.toString()); + } + return fileCacheValues; + } + + /** + * The key of hive partition values cache. + */ + @Data + public static class PartitionValueCacheKey { + private NameMapping nameMapping; + // Not part of cache identity. + private List types; + + public PartitionValueCacheKey(NameMapping nameMapping, List types) { + this.nameMapping = nameMapping; + this.types = types; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof PartitionValueCacheKey)) { + return false; + } + return nameMapping.equals(((PartitionValueCacheKey) obj).nameMapping); + } + + @Override + public int hashCode() { + return nameMapping.hashCode(); + } + + @Override + public String toString() { + return "PartitionValueCacheKey{" + "dbName='" + nameMapping.getLocalDbName() + '\'' + + ", tblName='" + nameMapping.getLocalTblName() + '\'' + '}'; + } + } + + @Data + public static class PartitionCacheKey { + private NameMapping nameMapping; + private List values; + + public PartitionCacheKey(NameMapping nameMapping, List values) { + this.nameMapping = nameMapping; + this.values = values; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof PartitionCacheKey)) { + return false; + } + return nameMapping.equals(((PartitionCacheKey) obj).nameMapping) + && Objects.equals(values, ((PartitionCacheKey) obj).values); + } + + boolean isSameTable(String dbName, String tblName) { + return this.nameMapping.getLocalDbName().equals(dbName) + && this.nameMapping.getLocalTblName().equals(tblName); + } + + @Override + public int hashCode() { + return Objects.hash(nameMapping, values); + } + + @Override + public String toString() { + return "PartitionCacheKey{" + "dbName='" + nameMapping.getLocalDbName() + '\'' + + ", tblName='" + nameMapping.getLocalTblName() + '\'' + ", values=" + values + '}'; + } + } + + @Data + public static class FileCacheKey { + private long dummyKey = 0; + private long catalogId; + private String location; + // Not part of cache identity. + private String inputFormat; + // The values of partitions. + protected List partitionValues; + private long id; + + public FileCacheKey(long catalogId, long id, String location, String inputFormat, + List partitionValues) { + this.catalogId = catalogId; + this.location = location; + this.inputFormat = inputFormat; + this.partitionValues = partitionValues == null ? Lists.newArrayList() : partitionValues; + this.id = id; + } + + public static FileCacheKey createDummyCacheKey(long catalogId, long id, String location, + String inputFormat) { + FileCacheKey fileCacheKey = new FileCacheKey(catalogId, id, location, inputFormat, null); + fileCacheKey.dummyKey = Objects.hash(catalogId, id); + return fileCacheKey; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof FileCacheKey)) { + return false; + } + if (dummyKey != 0) { + return dummyKey == ((FileCacheKey) obj).dummyKey; + } + return catalogId == ((FileCacheKey) obj).catalogId + && location.equals(((FileCacheKey) obj).location) + && Objects.equals(partitionValues, ((FileCacheKey) obj).partitionValues); + } + + boolean isSameTable(long id) { + return this.id == id; + } + + @Override + public int hashCode() { + if (dummyKey != 0) { + return Objects.hash(dummyKey); + } + return Objects.hash(catalogId, location, partitionValues); + } + + @Override + public String toString() { + return "FileCacheKey{" + "catalogId=" + catalogId + ", location='" + location + '\'' + + ", inputFormat='" + inputFormat + '\'' + '}'; + } + } + + @Data + public static class FileCacheValue { + private final List files = Lists.newArrayList(); + private boolean isSplittable; + protected List partitionValues; + private AcidInfo acidInfo; + + public void addFile(RemoteFile file, LocationPath locationPath) { + if (isFileVisible(file.getPath())) { + HiveFileStatus status = new HiveFileStatus(); + status.setBlockLocations(file.getBlockLocations()); + status.setPath(locationPath); + status.length = file.getSize(); + status.blockSize = file.getBlockSize(); + status.modificationTime = file.getModificationTime(); + files.add(status); + } + } + + public int getValuesSize() { + return partitionValues == null ? 0 : partitionValues.size(); + } + + @VisibleForTesting + public static boolean isFileVisible(Path path) { + if (path == null) { + return false; + } + String pathStr = path.toUri().toString(); + if (containsHiddenPath(pathStr)) { + return false; + } + return true; + } + + private static boolean containsHiddenPath(String path) { + if (path.startsWith(".") || path.startsWith("_")) { + return true; + } + for (int i = 0; i < path.length() - 1; i++) { + if (path.charAt(i) == '/' && (path.charAt(i + 1) == '.' || path.charAt(i + 1) == '_')) { + return true; + } + } + return false; + } + } + + @Data + public static class HiveFileStatus { + BlockLocation[] blockLocations; + LocationPath path; + long length; + long blockSize; + long modificationTime; + boolean splittable; + List partitionValues; + AcidInfo acidInfo; + } + + @Data + public static class HivePartitionValues { + private BiMap partitionNameToIdMap; + private Map idToPartitionItem; + private Map> partitionValuesMap; + + // Sorted partition ranges for binary search filtering. + private SortedPartitionRanges sortedPartitionRanges; + + public HivePartitionValues() { + } + + public HivePartitionValues(Map idToPartitionItem, + BiMap partitionNameToIdMap, + Map> partitionValuesMap) { + this.idToPartitionItem = idToPartitionItem; + this.partitionNameToIdMap = partitionNameToIdMap; + this.partitionValuesMap = partitionValuesMap; + this.sortedPartitionRanges = buildSortedPartitionRanges(); + } + + public HivePartitionValues copy() { + HivePartitionValues copy = new HivePartitionValues(); + copy.setPartitionNameToIdMap(partitionNameToIdMap == null ? null : HashBiMap.create(partitionNameToIdMap)); + copy.setIdToPartitionItem(idToPartitionItem == null ? null : Maps.newHashMap(idToPartitionItem)); + copy.setPartitionValuesMap(partitionValuesMap == null ? null : Maps.newHashMap(partitionValuesMap)); + return copy; + } + + public void rebuildSortedPartitionRanges() { + this.sortedPartitionRanges = buildSortedPartitionRanges(); + } + + public java.util.Optional> getSortedPartitionRanges() { + return java.util.Optional.ofNullable(sortedPartitionRanges); + } + + private SortedPartitionRanges buildSortedPartitionRanges() { + if (partitionNameToIdMap == null || partitionNameToIdMap.isEmpty() + || idToPartitionItem == null || idToPartitionItem.isEmpty()) { + return null; + } + + BiMap idToName = partitionNameToIdMap.inverse(); + Map nameToPartitionItem = Maps.newHashMapWithExpectedSize(idToPartitionItem.size()); + for (Map.Entry entry : idToPartitionItem.entrySet()) { + String partitionName = idToName.get(entry.getKey()); + if (partitionName != null) { + nameToPartitionItem.put(partitionName, entry.getValue()); + } + } + + return SortedPartitionRanges.build(nameToPartitionItem); + } + } +} 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 deleted file mode 100644 index 8b07f3e875a5ce..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java +++ /dev/null @@ -1,1064 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.datasource.hive; - -import org.apache.doris.analysis.PartitionValue; -import org.apache.doris.backup.Status.ErrCode; -import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.Env; -import org.apache.doris.catalog.ListPartitionItem; -import org.apache.doris.catalog.PartitionItem; -import org.apache.doris.catalog.PartitionKey; -import org.apache.doris.catalog.TableIf; -import org.apache.doris.catalog.Type; -import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.CacheFactory; -import org.apache.doris.common.Config; -import org.apache.doris.common.FeConstants; -import org.apache.doris.common.UserException; -import org.apache.doris.common.security.authentication.AuthenticationConfig; -import org.apache.doris.common.security.authentication.HadoopAuthenticator; -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.ExternalCatalog; -import org.apache.doris.datasource.ExternalMetaCacheMgr; -import org.apache.doris.datasource.ExternalTable; -import org.apache.doris.datasource.NameMapping; -import org.apache.doris.fs.DirectoryLister; -import org.apache.doris.fs.FileSystemCache; -import org.apache.doris.fs.FileSystemDirectoryLister; -import org.apache.doris.fs.FileSystemIOException; -import org.apache.doris.fs.RemoteIterator; -import org.apache.doris.fs.remote.RemoteFile; -import org.apache.doris.fs.remote.RemoteFileSystem; -import org.apache.doris.metric.GaugeMetric; -import org.apache.doris.metric.Metric; -import org.apache.doris.metric.MetricLabel; -import org.apache.doris.metric.MetricRepo; -import org.apache.doris.nereids.rules.expression.rules.SortedPartitionRanges; -import org.apache.doris.planner.ListPartitionPrunerV2; - -import com.github.benmanes.caffeine.cache.CacheLoader; -import com.github.benmanes.caffeine.cache.LoadingCache; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.base.Strings; -import com.google.common.collect.BiMap; -import com.google.common.collect.HashBiMap; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Streams; -import lombok.Data; -import org.apache.commons.lang3.math.NumberUtils; -import org.apache.hadoop.fs.BlockLocation; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.StorageDescriptor; -import org.apache.hadoop.hive.metastore.utils.FileUtils; -import org.apache.hadoop.mapred.JobConf; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.OptionalLong; -import java.util.Set; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; - -// The cache of a hms catalog. 3 kind of caches: -// 1. partitionValuesCache: cache the partition values of a table, for partition prune. -// 2. partitionCache: cache the partition info(location, input format, etc.) of a table. -// 3. fileCache: cache the files of a location. -public class HiveMetaStoreCache { - private static final Logger LOG = LogManager.getLogger(HiveMetaStoreCache.class); - public static final String HIVE_DEFAULT_PARTITION = "__HIVE_DEFAULT_PARTITION__"; - public static final String ERR_CACHE_INCONSISTENCY = "ERR_CACHE_INCONSISTENCY: "; - - private final HMSExternalCatalog catalog; - private JobConf jobConf; - private final ExecutorService refreshExecutor; - private final ExecutorService fileListingExecutor; - - // cache from -> - private LoadingCache partitionValuesCache; - // cache from -> - private LoadingCache partitionCache; - // the ref of cache from -> - // Other thread may reset this cache, so use AtomicReference to wrap it. - private volatile AtomicReference> fileCacheRef - = new AtomicReference<>(); - - public HiveMetaStoreCache(HMSExternalCatalog catalog, - ExecutorService refreshExecutor, ExecutorService fileListingExecutor) { - this.catalog = catalog; - this.refreshExecutor = refreshExecutor; - this.fileListingExecutor = fileListingExecutor; - init(); - initMetrics(); - } - - /** - * Because the partitionValuesCache|partitionCache|fileCache use the same executor for batch loading, - * we need to be very careful and try to avoid the circular dependency of these tasks - * which will bring out thread deadlock. - **/ - public void init() { - long partitionCacheTtlSecond = NumberUtils.toLong( - (catalog.getProperties().get(HMSExternalCatalog.PARTITION_CACHE_TTL_SECOND)), - ExternalCatalog.CACHE_NO_TTL); - - CacheFactory partitionValuesCacheFactory = new CacheFactory( - OptionalLong.of(partitionCacheTtlSecond >= ExternalCatalog.CACHE_TTL_DISABLE_CACHE - ? partitionCacheTtlSecond : Config.external_cache_expire_time_seconds_after_access), - OptionalLong.of(Config.external_cache_refresh_time_minutes * 60L), - Config.max_hive_partition_table_cache_num, - true, - null); - partitionValuesCache = partitionValuesCacheFactory.buildCache(this::loadPartitionValues, - refreshExecutor); - - CacheFactory partitionCacheFactory = new CacheFactory( - OptionalLong.of(Config.external_cache_expire_time_seconds_after_access), - OptionalLong.empty(), - Config.max_hive_partition_cache_num, - true, - null); - partitionCache = partitionCacheFactory.buildCache(new CacheLoader() { - @Override - public HivePartition load(PartitionCacheKey key) { - return loadPartition(key); - } - - @Override - public Map loadAll(Iterable keys) { - return loadPartitions(keys); - } - }, refreshExecutor); - - setNewFileCache(); - } - - /*** - * generate a filecache and set to fileCacheRef - */ - private void setNewFileCache() { - // if the file.meta.cache.ttl-second is equal or greater than 0, the cache expired will be set to that value - int fileMetaCacheTtlSecond = NumberUtils.toInt( - (catalog.getProperties().get(HMSExternalCatalog.FILE_META_CACHE_TTL_SECOND)), - ExternalCatalog.CACHE_NO_TTL); - - CacheFactory fileCacheFactory = new CacheFactory( - OptionalLong.of(fileMetaCacheTtlSecond >= ExternalCatalog.CACHE_TTL_DISABLE_CACHE - ? fileMetaCacheTtlSecond : Config.external_cache_expire_time_seconds_after_access), - OptionalLong.of(Config.external_cache_refresh_time_minutes * 60L), - Config.max_external_file_cache_num, - true, - null); - - CacheLoader loader = new CacheBulkLoader() { - @Override - protected ExecutorService getExecutor() { - return HiveMetaStoreCache.this.fileListingExecutor; - } - - @Override - public FileCacheValue load(FileCacheKey key) { - return loadFiles(key, new FileSystemDirectoryLister(), null); - } - }; - - LoadingCache oldFileCache = fileCacheRef.get(); - - fileCacheRef.set(fileCacheFactory.buildCache(loader, this.refreshExecutor)); - if (Objects.nonNull(oldFileCache)) { - oldFileCache.invalidateAll(); - } - } - - private void initMetrics() { - // partition value - GaugeMetric valueCacheGauge = new GaugeMetric("hive_meta_cache", - Metric.MetricUnit.NOUNIT, "hive partition value cache number") { - @Override - public Long getValue() { - return partitionValuesCache.estimatedSize(); - } - }; - valueCacheGauge.addLabel(new MetricLabel("type", "partition_value")); - valueCacheGauge.addLabel(new MetricLabel("catalog", catalog.getName())); - MetricRepo.DORIS_METRIC_REGISTER.addMetrics(valueCacheGauge); - // partition - GaugeMetric partitionCacheGauge = new GaugeMetric("hive_meta_cache", - Metric.MetricUnit.NOUNIT, "hive partition cache number") { - @Override - public Long getValue() { - return partitionCache.estimatedSize(); - } - }; - partitionCacheGauge.addLabel(new MetricLabel("type", "partition")); - partitionCacheGauge.addLabel(new MetricLabel("catalog", catalog.getName())); - MetricRepo.DORIS_METRIC_REGISTER.addMetrics(partitionCacheGauge); - // file - GaugeMetric fileCacheGauge = new GaugeMetric("hive_meta_cache", - Metric.MetricUnit.NOUNIT, "hive file cache number") { - @Override - public Long getValue() { - return fileCacheRef.get().estimatedSize(); - } - }; - fileCacheGauge.addLabel(new MetricLabel("type", "file")); - fileCacheGauge.addLabel(new MetricLabel("catalog", catalog.getName())); - MetricRepo.DORIS_METRIC_REGISTER.addMetrics(fileCacheGauge); - } - - private HivePartitionValues loadPartitionValues(PartitionValueCacheKey key) { - // partition name format: nation=cn/city=beijing,`listPartitionNames` returned string is the encoded string. - NameMapping nameMapping = key.nameMapping; - List partitionNames = catalog.getClient() - .listPartitionNames(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName()); - if (LOG.isDebugEnabled()) { - LOG.debug("load #{} partitions for {} in catalog {}", partitionNames.size(), key, catalog.getName()); - } - Map idToPartitionItem = Maps.newHashMapWithExpectedSize(partitionNames.size()); - BiMap partitionNameToIdMap = HashBiMap.create(partitionNames.size()); - for (String partitionName : partitionNames) { - long partitionId = Util.genIdByName(catalog.getName(), nameMapping.getLocalDbName(), - nameMapping.getLocalTblName(), partitionName); - ListPartitionItem listPartitionItem = toListPartitionItem(partitionName, key.types); - idToPartitionItem.put(partitionId, listPartitionItem); - partitionNameToIdMap.put(partitionName, partitionId); - } - - Map> partitionValuesMap = ListPartitionPrunerV2.getPartitionValuesMap(idToPartitionItem); - return new HivePartitionValues(idToPartitionItem, partitionNameToIdMap, partitionValuesMap); - } - - private ListPartitionItem toListPartitionItem(String partitionName, List types) { - // Partition name will be in format: nation=cn/city=beijing - // parse it to get values "cn" and "beijing" - List partitionValues = HiveUtil.toPartitionValues(partitionName); - Preconditions.checkState(partitionValues.size() == types.size(), - ERR_CACHE_INCONSISTENCY + partitionName + " vs. " + types); - List values = Lists.newArrayListWithExpectedSize(types.size()); - for (String partitionValue : partitionValues) { - values.add(new PartitionValue(partitionValue, HIVE_DEFAULT_PARTITION.equals(partitionValue))); - } - try { - PartitionKey key = PartitionKey.createListPartitionKeyWithTypes(values, types, true); - ListPartitionItem listPartitionItem = new ListPartitionItem(Lists.newArrayList(key)); - return listPartitionItem; - } catch (AnalysisException e) { - throw new CacheException("failed to convert hive partition %s to list partition in catalog %s", - e, partitionName, catalog.getName()); - } - } - - private HivePartition loadPartition(PartitionCacheKey key) { - NameMapping nameMapping = key.nameMapping; - Partition partition = catalog.getClient() - .getPartition(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName(), key.values); - StorageDescriptor sd = partition.getSd(); - if (LOG.isDebugEnabled()) { - LOG.debug("load partition format: {}, location: {} for {} in catalog {}", - sd.getInputFormat(), sd.getLocation(), key, catalog.getName()); - } - // TODO: more info? - return new HivePartition(nameMapping, false, sd.getInputFormat(), sd.getLocation(), key.values, - partition.getParameters()); - } - - private Map loadPartitions(Iterable keys) { - Map ret = new HashMap<>(); - if (keys == null || !keys.iterator().hasNext()) { - return ret; - } - // The name mapping is same for all keys, so we can just get one key to get the name mapping. - PartitionCacheKey oneKey = Iterables.get(keys, 0); - NameMapping nameMapping = oneKey.nameMapping; - String localDbName = nameMapping.getLocalDbName(); - String localTblName = nameMapping.getLocalTblName(); - List partitionColumns = ((HMSExternalTable) - (catalog.getDbNullable(localDbName).getTableNullable(localTblName))).getPartitionColumns(); - // A partitionName is like "country=China/city=Beijing" or "date=2023-02-01" - List partitionNames = Streams.stream(keys).map(key -> { - StringBuilder sb = new StringBuilder(); - Preconditions.checkState(key.getValues().size() == partitionColumns.size()); - for (int i = 0; i < partitionColumns.size(); i++) { - // Partition name and value may contain special character, like / and so on. Need to encode. - sb.append(FileUtils.escapePathName(partitionColumns.get(i).getName())); - sb.append("="); - sb.append(FileUtils.escapePathName(key.getValues().get(i))); - sb.append("/"); - } - sb.delete(sb.length() - 1, sb.length()); - return sb.toString(); - }).collect(Collectors.toList()); - - - List partitions = catalog.getClient() - .getPartitions(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName(), partitionNames); - // Compose the return result map. - for (Partition partition : partitions) { - StorageDescriptor sd = partition.getSd(); - ret.put(new PartitionCacheKey(nameMapping, partition.getValues()), - new HivePartition(nameMapping, false, - sd.getInputFormat(), sd.getLocation(), partition.getValues(), partition.getParameters())); - } - return ret; - } - - // Get File Status by using FileSystem API. - private FileCacheValue getFileCache(LocationPath path, String inputFormat, - List partitionValues, - DirectoryLister directoryLister, - TableIf table) throws UserException { - FileCacheValue result = new FileCacheValue(); - - FileSystemCache.FileSystemCacheKey fileSystemCacheKey = new FileSystemCache.FileSystemCacheKey( - path.getFsIdentifier(), path.getStorageProperties() - ); - RemoteFileSystem fs = Env.getCurrentEnv().getExtMetaCacheMgr().getFsCache() - .getRemoteFileSystem(fileSystemCacheKey); - result.setSplittable(HiveUtil.isSplittable(fs, inputFormat, path.getNormalizedLocation())); - // For Tez engine, it may generate subdirectoies for "union" query. - // So there may be files and directories in the table directory at the same time. eg: - // /user/hive/warehouse/region_tmp_union_all2/000000_0 - // /user/hive/warehouse/region_tmp_union_all2/1 - // /user/hive/warehouse/region_tmp_union_all2/2 - // So we need to recursively list data location. - // https://blog.actorsfit.com/a?ID=00550-ce56ec63-1bff-4b0c-a6f7-447b93efaa31 - boolean isRecursiveDirectories = Boolean.valueOf( - catalog.getProperties().getOrDefault("hive.recursive_directories", "true")); - try { - RemoteIterator iterator = directoryLister.listFiles(fs, isRecursiveDirectories, - table, path.getNormalizedLocation()); - while (iterator.hasNext()) { - RemoteFile remoteFile = iterator.next(); - String srcPath = remoteFile.getPath().toString(); - LocationPath locationPath = LocationPath.of(srcPath, path.getStorageProperties()); - result.addFile(remoteFile, locationPath); - } - } catch (FileSystemIOException e) { - if (e.getErrorCode().isPresent() && e.getErrorCode().get().equals(ErrCode.NOT_FOUND)) { - // User may manually remove partition under HDFS, in this case, - // Hive doesn't aware that the removed partition is missing. - // Here is to support this case without throw an exception. - LOG.warn(String.format("File %s not exist.", path.getNormalizedLocation())); - if (!Boolean.valueOf(catalog.getProperties() - .getOrDefault("hive.ignore_absent_partitions", "true"))) { - throw new UserException("Partition location does not exist: " + path.getNormalizedLocation()); - } - } else { - throw new RuntimeException(e); - } - } - // Must copy the partitionValues to avoid concurrent modification of key and value - result.setPartitionValues(Lists.newArrayList(partitionValues)); - return result; - } - - private FileCacheValue loadFiles(FileCacheKey key, DirectoryLister directoryLister, TableIf table) { - ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - try { - Thread.currentThread().setContextClassLoader(ClassLoader.getSystemClassLoader()); - LocationPath finalLocation = LocationPath.of(key.getLocation(), catalog.getCatalogProperty() - .getStoragePropertiesMap()); - // NOTICE: the setInputPaths has 2 overloads, the 2nd arg should be Path not String - try { - FileCacheValue result = getFileCache(finalLocation, key.inputFormat, - key.getPartitionValues(), directoryLister, table); - // Replace default hive partition with a null_string. - for (int i = 0; i < result.getValuesSize(); i++) { - if (HIVE_DEFAULT_PARTITION.equals(result.getPartitionValues().get(i))) { - result.getPartitionValues().set(i, FeConstants.null_string); - } - } - - if (LOG.isDebugEnabled()) { - LOG.debug("load #{} splits for {} in catalog {}", result.getFiles().size(), key, catalog.getName()); - } - return result; - } catch (Exception e) { - throw new CacheException("failed to get input splits for %s in catalog %s", e, key, catalog.getName()); - } - } finally { - Thread.currentThread().setContextClassLoader(classLoader); - } - } - - public HivePartitionValues getPartitionValues(ExternalTable dorisTable, List types) { - PartitionValueCacheKey key = new PartitionValueCacheKey(dorisTable.getOrBuildNameMapping(), types); - return getPartitionValues(key); - } - - @VisibleForTesting - public HivePartitionValues getPartitionValues(PartitionValueCacheKey key) { - return partitionValuesCache.get(key); - } - - public List getFilesByPartitions(List partitions, - boolean withCache, - boolean concurrent, - DirectoryLister directoryLister, - TableIf table) { - long start = System.currentTimeMillis(); - if (partitions.isEmpty()) { - return Lists.newArrayList(); - } - HivePartition firstPartition = partitions.get(0); - long fileId = Util.genIdByName(firstPartition.getNameMapping().getLocalDbName(), - firstPartition.getNameMapping().getLocalTblName()); - List keys = partitions.stream().map(p -> p.isDummyPartition() - ? FileCacheKey.createDummyCacheKey( - fileId, p.getPath(), p.getInputFormat()) - : new FileCacheKey(fileId, p.getPath(), - p.getInputFormat(), p.getPartitionValues())) - .collect(Collectors.toList()); - - List fileLists; - try { - if (withCache) { - fileLists = new ArrayList<>(fileCacheRef.get().getAll(keys).values()); - } else { - if (concurrent) { - List> pList = keys.stream().map( - key -> fileListingExecutor.submit(() -> loadFiles(key, directoryLister, table))) - .collect(Collectors.toList()); - fileLists = Lists.newArrayListWithExpectedSize(keys.size()); - for (Future p : pList) { - fileLists.add(p.get()); - } - } else { - fileLists = keys.stream().map((key) -> loadFiles(key, directoryLister, table)) - .collect(Collectors.toList()); - } - } - } catch (ExecutionException e) { - throw new CacheException("failed to get files from partitions in catalog %s", - e, catalog.getName()); - } catch (InterruptedException e) { - throw new CacheException("failed to get files from partitions in catalog %s with interrupted exception", - e, catalog.getName()); - } - - if (LOG.isDebugEnabled()) { - LOG.debug("get #{} files from #{} partitions in catalog {} cost: {} ms", - fileLists.stream().mapToInt(l -> l.getFiles() == null ? 0 : l.getFiles().size()).sum(), - partitions.size(), catalog.getName(), (System.currentTimeMillis() - start)); - } - return fileLists; - } - - public HivePartition getHivePartition(ExternalTable dorisTable, List partitionValues) { - return partitionCache.get(new PartitionCacheKey(dorisTable.getOrBuildNameMapping(), partitionValues)); - } - - public List getAllPartitionsWithCache(ExternalTable dorisTable, - List> partitionValuesList) { - return getAllPartitions(dorisTable, partitionValuesList, true); - } - - public List getAllPartitionsWithoutCache(ExternalTable dorisTable, - List> partitionValuesList) { - return getAllPartitions(dorisTable, partitionValuesList, false); - } - - private List getAllPartitions(ExternalTable dorisTable, List> partitionValuesList, - boolean withCache) { - long start = System.currentTimeMillis(); - NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); - List keys = partitionValuesList.stream() - .map(p -> new PartitionCacheKey(nameMapping, p)) - .collect(Collectors.toList()); - - List partitions; - if (withCache) { - partitions = partitionCache.getAll(keys).values().stream().collect(Collectors.toList()); - } else { - Map map = loadPartitions(keys); - partitions = map.values().stream().collect(Collectors.toList()); - } - - if (LOG.isDebugEnabled()) { - LOG.debug("get #{} partitions in catalog {} cost: {} ms", partitions.size(), catalog.getName(), - (System.currentTimeMillis() - start)); - } - return partitions; - } - - public void invalidateTableCache(NameMapping nameMapping) { - partitionValuesCache.invalidate(new PartitionValueCacheKey(nameMapping, null)); - partitionCache.asMap().keySet().forEach(k -> { - if (k.isSameTable(nameMapping.getLocalDbName(), nameMapping.getLocalTblName())) { - partitionCache.invalidate(k); - } - }); - long id = Util.genIdByName(nameMapping.getLocalDbName(), nameMapping.getLocalTblName()); - LoadingCache fileCache = fileCacheRef.get(); - fileCache.asMap().keySet().forEach(k -> { - if (k.isSameTable(id)) { - fileCache.invalidate(k); - } - }); - } - - public void invalidatePartitionCache(ExternalTable dorisTable, String partitionName) { - NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); - long id = Util.genIdByName(nameMapping.getLocalDbName(), nameMapping.getLocalTblName()); - PartitionValueCacheKey key = new PartitionValueCacheKey(nameMapping, null); - HivePartitionValues partitionValues = partitionValuesCache.getIfPresent(key); - if (partitionValues != null) { - Long partitionId = partitionValues.partitionNameToIdMap.get(partitionName); - List values = partitionValues.partitionValuesMap.get(partitionId); - PartitionCacheKey partKey = new PartitionCacheKey(nameMapping, values); - HivePartition partition = partitionCache.getIfPresent(partKey); - if (partition != null) { - fileCacheRef.get().invalidate(new FileCacheKey(id, partition.getPath(), - null, partition.getPartitionValues())); - partitionCache.invalidate(partKey); - } - } - } - - /** - * Selectively refreshes cache for affected partitions based on update information from BE. - * For APPEND/OVERWRITE: invalidate both partition cache and file cache using existing method. - * For NEW: add to partition values cache. - * - * @param table The Hive table whose partitions were modified - * @param partitionUpdates List of partition updates from BE - * @param modifiedPartNames Output list to collect names of modified partitions - * @param newPartNames Output list to collect names of new partitions - */ - public void refreshAffectedPartitions(HMSExternalTable table, - List partitionUpdates, - List modifiedPartNames, List newPartNames) { - if (partitionUpdates == null || partitionUpdates.isEmpty()) { - return; - } - - for (org.apache.doris.thrift.THivePartitionUpdate update : partitionUpdates) { - String partitionName = update.getName(); - // Skip if partition name is null/empty (non-partitioned table case) - if (Strings.isNullOrEmpty(partitionName)) { - continue; - } - - switch (update.getUpdateMode()) { - case APPEND: - case OVERWRITE: - modifiedPartNames.add(partitionName); - break; - case NEW: - newPartNames.add(partitionName); - break; - default: - LOG.warn("Unknown update mode {} for partition {}", - update.getUpdateMode(), partitionName); - break; - } - } - - refreshAffectedPartitionsCache(table, modifiedPartNames, newPartNames); - } - - public void refreshAffectedPartitionsCache(HMSExternalTable table, - List modifiedPartNames, List newPartNames) { - - // Invalidate cache for modified partitions (both partition cache and file cache) - for (String partitionName : modifiedPartNames) { - invalidatePartitionCache(table, partitionName); - } - // Merge modifiedPartNames and newPartNames - // Case: - // 1. hive, insert into a new partition p_new - // 2. doris-observer, insert into same partition p_new - // 1. forward insert command to Master - // 2. Master FE will refresh its cache and get p_new into its partition values cache - // 3. Insert finished and Master write edit log, but p_new is recorded as MODIFIED not NEW. - // (See refreshAffectedPartitions() methods) - // 4. Observer FE receive edit log and refresh cache, if we don't merge them, - // it will miss adding p_new to its partition values cache. - List mergedPartNames = Lists.newArrayList(modifiedPartNames); - mergedPartNames.addAll(newPartNames); - // Add new partitions to partition values cache - if (!mergedPartNames.isEmpty()) { - addPartitionsCache(table.getOrBuildNameMapping(), mergedPartNames, - table.getPartitionColumnTypes(Optional.empty())); - } - - // Log summary - LOG.info("Refreshed cache for table {}: {} modified partitions, {} new partitions", - table.getName(), modifiedPartNames.size(), newPartNames.size()); - } - - public void invalidateDbCache(String dbName) { - long start = System.currentTimeMillis(); - Set keys = partitionValuesCache.asMap().keySet(); - for (PartitionValueCacheKey key : keys) { - if (key.nameMapping.getLocalDbName().equals(dbName)) { - invalidateTableCache(key.nameMapping); - } - } - if (LOG.isDebugEnabled()) { - LOG.debug("invalid db cache for {} in catalog {}, cache num: {}, cost: {} ms", dbName, catalog.getName(), - keys.size(), (System.currentTimeMillis() - start)); - } - } - - public void invalidateAll() { - partitionValuesCache.invalidateAll(); - partitionCache.invalidateAll(); - fileCacheRef.get().invalidateAll(); - if (LOG.isDebugEnabled()) { - LOG.debug("invalid all meta cache in catalog {}", catalog.getName()); - } - } - - // partition name format: nation=cn/city=beijing - public void addPartitionsCache(NameMapping nameMapping, List partitionNames, - List partitionColumnTypes) { - PartitionValueCacheKey key = new PartitionValueCacheKey(nameMapping, partitionColumnTypes); - HivePartitionValues partitionValues = partitionValuesCache.getIfPresent(key); - if (partitionValues == null) { - return; - } - HivePartitionValues copy = partitionValues.copy(); - Map idToPartitionItemBefore = copy.getIdToPartitionItem(); - Map partitionNameToIdMapBefore = copy.getPartitionNameToIdMap(); - Map idToPartitionItem = new HashMap<>(); - String localDbName = nameMapping.getLocalDbName(); - String localTblName = nameMapping.getLocalTblName(); - for (String partitionName : partitionNames) { - if (partitionNameToIdMapBefore.containsKey(partitionName)) { - LOG.info("addPartitionsCache partitionName:[{}] has exist in table:[{}]", partitionName, localTblName); - continue; - } - long partitionId = Util.genIdByName(catalog.getName(), localDbName, localTblName, partitionName); - ListPartitionItem listPartitionItem = toListPartitionItem(partitionName, key.types); - idToPartitionItemBefore.put(partitionId, listPartitionItem); - idToPartitionItem.put(partitionId, listPartitionItem); - partitionNameToIdMapBefore.put(partitionName, partitionId); - } - Map> partitionValuesMapBefore = copy.getPartitionValuesMap(); - Map> partitionValuesMap = ListPartitionPrunerV2.getPartitionValuesMap(idToPartitionItem); - partitionValuesMapBefore.putAll(partitionValuesMap); - // Rebuild sorted partition ranges after adding partitions - copy.rebuildSortedPartitionRanges(); - HivePartitionValues partitionValuesCur = partitionValuesCache.getIfPresent(key); - if (partitionValuesCur == partitionValues) { - partitionValuesCache.put(key, copy); - } - } - - public void dropPartitionsCache(ExternalTable dorisTable, List partitionNames, - boolean invalidPartitionCache) { - NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); - PartitionValueCacheKey key = new PartitionValueCacheKey(nameMapping, null); - HivePartitionValues partitionValues = partitionValuesCache.getIfPresent(key); - if (partitionValues == null) { - return; - } - HivePartitionValues copy = partitionValues.copy(); - Map partitionNameToIdMapBefore = copy.getPartitionNameToIdMap(); - Map idToPartitionItemBefore = copy.getIdToPartitionItem(); - Map> partitionValuesMap = copy.getPartitionValuesMap(); - for (String partitionName : partitionNames) { - if (!partitionNameToIdMapBefore.containsKey(partitionName)) { - LOG.info("dropPartitionsCache partitionName:[{}] not exist in table:[{}]", - partitionName, nameMapping.getFullLocalName()); - continue; - } - Long partitionId = partitionNameToIdMapBefore.remove(partitionName); - idToPartitionItemBefore.remove(partitionId); - partitionValuesMap.remove(partitionId); - - if (invalidPartitionCache) { - invalidatePartitionCache(dorisTable, partitionName); - } - } - // Rebuild sorted partition ranges after dropping partitions - copy.rebuildSortedPartitionRanges(); - HivePartitionValues partitionValuesCur = partitionValuesCache.getIfPresent(key); - if (partitionValuesCur == partitionValues) { - partitionValuesCache.put(key, copy); - } - } - - public void putPartitionValuesCacheForTest(PartitionValueCacheKey key, HivePartitionValues values) { - partitionValuesCache.put(key, values); - } - - /*** - * get fileCache ref - * @return - */ - @VisibleForTesting - public AtomicReference> getFileCacheRef() { - return fileCacheRef; - } - - @VisibleForTesting - public LoadingCache getPartitionValuesCache() { - return partitionValuesCache; - } - - @VisibleForTesting - public LoadingCache getPartitionCache() { - return partitionCache; - } - - public List getFilesByTransaction(List partitions, Map txnValidIds, - boolean isFullAcid, String bindBrokerName) { - List fileCacheValues = Lists.newArrayList(); - try { - if (partitions.isEmpty()) { - return fileCacheValues; - } - for (HivePartition partition : partitions) { - //Get filesystem multiple times, Reason: https://github.com/apache/doris/pull/23409. - LocationPath locationPath = LocationPath.of(partition.getPath(), - catalog.getCatalogProperty().getStoragePropertiesMap()); - // Use the bind broker name to get the file system, so that the file system can be shared - RemoteFileSystem fileSystem = Env.getCurrentEnv().getExtMetaCacheMgr().getFsCache().getRemoteFileSystem( - new FileSystemCache.FileSystemCacheKey( - locationPath.getNormalizedLocation(), - locationPath.getStorageProperties())); - // consider other methods to get the authenticator - AuthenticationConfig authenticationConfig = AuthenticationConfig.getKerberosConfig(locationPath - .getStorageProperties().getBackendConfigProperties()); - HadoopAuthenticator hadoopAuthenticator = - HadoopAuthenticator.getHadoopAuthenticator(authenticationConfig); - - fileCacheValues.add( - hadoopAuthenticator.doAs(() -> AcidUtil.getAcidState( - fileSystem, partition, txnValidIds, catalog.getCatalogProperty() - .getStoragePropertiesMap(), isFullAcid)) - ); - } - } catch (Exception e) { - throw new CacheException("Failed to get input splits %s", e, txnValidIds.toString()); - } - return fileCacheValues; - } - - /** - * The Key of hive partition value cache - */ - @Data - public static class PartitionValueCacheKey { - private NameMapping nameMapping; - // not in key - private List types; - - public PartitionValueCacheKey(NameMapping nameMapping, List types) { - this.nameMapping = nameMapping; - this.types = types; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (!(obj instanceof PartitionValueCacheKey)) { - return false; - } - return nameMapping.equals(((PartitionValueCacheKey) obj).nameMapping); - } - - @Override - public int hashCode() { - return nameMapping.hashCode(); - } - - @Override - public String toString() { - return "PartitionValueCacheKey{" + "dbName='" + nameMapping.getLocalDbName() + '\'' + ", tblName='" - + nameMapping.getLocalTblName() + '\'' + '}'; - } - } - - @Data - public static class PartitionCacheKey { - private NameMapping nameMapping; - private List values; - - public PartitionCacheKey(NameMapping nameMapping, List values) { - this.nameMapping = nameMapping; - this.values = values; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (!(obj instanceof PartitionCacheKey)) { - return false; - } - return nameMapping.equals(((PartitionCacheKey) obj).nameMapping) - && Objects.equals(values, ((PartitionCacheKey) obj).values); - } - - boolean isSameTable(String dbName, String tblName) { - return this.nameMapping.getLocalDbName().equals(dbName) - && this.nameMapping.getLocalTblName().equals(tblName); - } - - @Override - public int hashCode() { - return Objects.hash(nameMapping, values); - } - - @Override - public String toString() { - return "PartitionCacheKey{" + "dbName='" + nameMapping.getLocalDbName() + '\'' - + ", tblName='" + nameMapping.getLocalTblName() + '\'' + ", values=" - + values + '}'; - } - } - - @Data - public static class FileCacheKey { - private long dummyKey = 0; - private String location; - // not in key - private String inputFormat; - // The values of partitions. - // e.g for file : hdfs://path/to/table/part1=a/part2=b/datafile - // partitionValues would be ["part1", "part2"] - protected List partitionValues; - private long id; - - public FileCacheKey(long id, String location, String inputFormat, - List partitionValues) { - this.location = location; - this.inputFormat = inputFormat; - this.partitionValues = partitionValues == null ? Lists.newArrayList() : partitionValues; - this.id = id; - } - - public static FileCacheKey createDummyCacheKey(long id, String location, - String inputFormat) { - FileCacheKey fileCacheKey = new FileCacheKey(id, location, inputFormat, null); - fileCacheKey.dummyKey = id; - return fileCacheKey; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (!(obj instanceof FileCacheKey)) { - return false; - } - if (dummyKey != 0) { - return dummyKey == ((FileCacheKey) obj).dummyKey; - } - return location.equals(((FileCacheKey) obj).location) - && Objects.equals(partitionValues, ((FileCacheKey) obj).partitionValues); - } - - boolean isSameTable(long id) { - return this.id == id; - } - - @Override - public int hashCode() { - if (dummyKey != 0) { - return Objects.hash(dummyKey); - } - return Objects.hash(location, partitionValues); - } - - @Override - public String toString() { - return "FileCacheKey{" + "location='" + location + '\'' + ", inputFormat='" + inputFormat + '\'' + '}'; - } - } - - @Data - public static class FileCacheValue { - // File Cache for self splitter. - private final List files = Lists.newArrayList(); - private boolean isSplittable; - // The values of partitions. - // e.g for file : hdfs://path/to/table/part1=a/part2=b/datafile - // partitionValues would be ["part1", "part2"] - protected List partitionValues; - - private AcidInfo acidInfo; - - public void addFile(RemoteFile file, LocationPath locationPath) { - if (isFileVisible(file.getPath())) { - HiveFileStatus status = new HiveFileStatus(); - status.setBlockLocations(file.getBlockLocations()); - status.setPath(locationPath); - status.length = file.getSize(); - status.blockSize = file.getBlockSize(); - status.modificationTime = file.getModificationTime(); - files.add(status); - } - } - - public int getValuesSize() { - return partitionValues == null ? 0 : partitionValues.size(); - } - - public AcidInfo getAcidInfo() { - return acidInfo; - } - - public void setAcidInfo(AcidInfo acidInfo) { - this.acidInfo = acidInfo; - } - - @VisibleForTesting - public static boolean isFileVisible(Path path) { - if (path == null) { - return false; - } - String pathStr = path.toUri().toString(); - if (containsHiddenPath(pathStr)) { - return false; - } - return true; - } - - private static boolean containsHiddenPath(String path) { - // Hive ignores files starting with _ and . - if (path.startsWith(".") || path.startsWith("_")) { - return true; - } - for (int i = 0; i < path.length() - 1; i++) { - if (path.charAt(i) == '/' && (path.charAt(i + 1) == '.' || path.charAt(i + 1) == '_')) { - return true; - } - } - return false; - } - } - - @Data - public static class HiveFileStatus { - BlockLocation[] blockLocations; - LocationPath path; - long length; - long blockSize; - long modificationTime; - boolean splittable; - List partitionValues; - AcidInfo acidInfo; - } - - @Data - public static class HivePartitionValues { - private BiMap partitionNameToIdMap; - private Map idToPartitionItem; - private Map> partitionValuesMap; - - // Sorted partition ranges for binary search filtering. - // Built at construction time, shares the same lifecycle with HivePartitionValues. - private SortedPartitionRanges sortedPartitionRanges; - - public HivePartitionValues() { - } - - public HivePartitionValues(Map idToPartitionItem, - BiMap partitionNameToIdMap, - Map> partitionValuesMap) { - this.idToPartitionItem = idToPartitionItem; - this.partitionNameToIdMap = partitionNameToIdMap; - this.partitionValuesMap = partitionValuesMap; - this.sortedPartitionRanges = buildSortedPartitionRanges(); - } - - /** - * Create a copy for incremental updates (add/drop partitions). - * The sortedPartitionRanges will be rebuilt after the caller modifies the partition data. - */ - public HivePartitionValues copy() { - HivePartitionValues copy = new HivePartitionValues(); - copy.setPartitionNameToIdMap(partitionNameToIdMap == null ? null : HashBiMap.create(partitionNameToIdMap)); - copy.setIdToPartitionItem(idToPartitionItem == null ? null : Maps.newHashMap(idToPartitionItem)); - copy.setPartitionValuesMap(partitionValuesMap == null ? null : Maps.newHashMap(partitionValuesMap)); - // sortedPartitionRanges is not copied here, caller should call rebuildSortedPartitionRanges() - // after modifying partition data - return copy; - } - - /** - * Rebuild sorted partition ranges after incremental updates. - * Should be called after add/drop partitions. - */ - public void rebuildSortedPartitionRanges() { - this.sortedPartitionRanges = buildSortedPartitionRanges(); - } - - /** - * Get sorted partition ranges for binary search filtering. - */ - public Optional> getSortedPartitionRanges() { - return Optional.ofNullable(sortedPartitionRanges); - } - - private SortedPartitionRanges buildSortedPartitionRanges() { - if (partitionNameToIdMap == null || partitionNameToIdMap.isEmpty()) { - return null; - } - - // Build name to partition item map for SortedPartitionRanges.buildFrom - BiMap idToName = partitionNameToIdMap.inverse(); - Map nameToPartitionItem = Maps.newHashMapWithExpectedSize(idToPartitionItem.size()); - for (Map.Entry entry : idToPartitionItem.entrySet()) { - String partitionName = idToName.get(entry.getKey()); - nameToPartitionItem.put(partitionName, entry.getValue()); - } - - return SortedPartitionRanges.build(nameToPartitionItem); - } - } - - /** - * get cache stats - * @return metric value>> - */ - public Map> getStats() { - Map> 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/hive/HudiDlaTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HudiDlaTable.java index 89f0ce8dc1b485..ebc374fa32c2fc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HudiDlaTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HudiDlaTable.java @@ -23,7 +23,6 @@ import org.apache.doris.catalog.PartitionType; import org.apache.doris.common.AnalysisException; import org.apache.doris.datasource.CacheException; -import org.apache.doris.datasource.ExternalSchemaCache; import org.apache.doris.datasource.SchemaCacheValue; import org.apache.doris.datasource.TablePartitionValues; import org.apache.doris.datasource.hudi.HudiMvccSnapshot; @@ -121,9 +120,9 @@ public HMSSchemaCacheValue getHudiSchemaCacheValue(Optional snapsh } private HMSSchemaCacheValue getHudiSchemaCacheValue(long timestamp) { - ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(hmsTable.getCatalog()); - Optional schemaCacheValue = cache.getSchemaValue( - new HudiSchemaCacheKey(hmsTable.getOrBuildNameMapping(), timestamp)); + Optional schemaCacheValue = Env.getCurrentEnv().getExtMetaCacheMgr() + .getSchemaCacheValue(hmsTable, + new HudiSchemaCacheKey(hmsTable.getOrBuildNameMapping(), timestamp)); if (!schemaCacheValue.isPresent()) { throw new CacheException("failed to getSchema for: %s.%s.%s.%s", null, hmsTable.getCatalog().getName(), hmsTable.getDbName(), hmsTable.getName(), timestamp); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java index 39bde49141856e..3d3a94eb4e19cf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java @@ -58,7 +58,7 @@ * Metastore clients like Apache Hive or Apache Spark configured to talk with the same metastore. *

* This class is used to poll metastore for such events at a given frequency. By observing - * such events, we can take appropriate action on the {@link org.apache.doris.datasource.hive.HiveMetaStoreCache} + * such events, we can take appropriate action on the {@link org.apache.doris.datasource.hive.HiveExternalMetaCache} * (refresh/invalidate/add/remove) so that represents the latest information * available in metastore. We keep track of the last synced event id in each polling * iteration so the next batch can be requested appropriately. The current batch size is diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java index 4454d48d74c52b..bed07db31bdb5c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java @@ -37,8 +37,8 @@ import org.apache.doris.datasource.hive.AcidInfo.DeleteDeltaInfo; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalTable; -import org.apache.doris.datasource.hive.HiveMetaStoreCache; -import org.apache.doris.datasource.hive.HiveMetaStoreCache.FileCacheValue; +import org.apache.doris.datasource.hive.HiveExternalMetaCache; +import org.apache.doris.datasource.hive.HiveExternalMetaCache.FileCacheValue; import org.apache.doris.datasource.hive.HiveMetaStoreClientHelper; import org.apache.doris.datasource.hive.HivePartition; import org.apache.doris.datasource.hive.HiveProperties; @@ -141,8 +141,8 @@ protected void doInitialize() throws UserException { protected List getPartitions() throws AnalysisException { List resPartitions = Lists.newArrayList(); - HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getMetaStoreCache((HMSExternalCatalog) hmsTable.getCatalog()); + HiveExternalMetaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() + .hive(hmsTable.getCatalog().getId()); List partitionColumnTypes = hmsTable.getPartitionColumnTypes(MvccUtil.getSnapshotFromContext(hmsTable)); if (!partitionColumnTypes.isEmpty()) { // partitioned table @@ -185,8 +185,8 @@ public List getSplits(int numBackends) throws UserException { prunedPartitions = getPartitions(); partitionInit = true; } - HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getMetaStoreCache((HMSExternalCatalog) hmsTable.getCatalog()); + HiveExternalMetaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() + .hive(hmsTable.getCatalog().getId()); String bindBrokerName = hmsTable.getCatalog().bindBrokerName(); List allFiles = Lists.newArrayList(); getFileSplitByPartitions(cache, prunedPartitions, allFiles, bindBrokerName, numBackends, false); @@ -213,8 +213,8 @@ public void startSplit(int numBackends) { splitAssignment.finishSchedule(); return; } - HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getMetaStoreCache((HMSExternalCatalog) hmsTable.getCatalog()); + HiveExternalMetaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() + .hive(hmsTable.getCatalog().getId()); Executor scheduleExecutor = Env.getCurrentEnv().getExtMetaCacheMgr().getScheduleExecutor(); String bindBrokerName = hmsTable.getCatalog().bindBrokerName(); AtomicInteger numFinishedPartitions = new AtomicInteger(0); @@ -280,7 +280,7 @@ public int numApproximateSplits() { return numSplitsPerPartition.get() * prunedPartitions.size(); } - private void getFileSplitByPartitions(HiveMetaStoreCache cache, List partitions, + private void getFileSplitByPartitions(HiveExternalMetaCache cache, List partitions, List allFiles, String bindBrokerName, int numBackends, boolean isBatchMode) throws IOException, UserException { List fileCaches; @@ -301,7 +301,7 @@ private void getFileSplitByPartitions(HiveMetaStoreCache cache, List hiveFileStatuses = selectFiles(fileCaches); + List hiveFileStatuses = selectFiles(fileCaches); splitAllFiles(allFiles, hiveFileStatuses, targetFileSplitSize); return; } @@ -327,13 +327,13 @@ private void getFileSplitByPartitions(HiveMetaStoreCache cache, List fileCaches, long result = sessionVariable.getMaxInitialSplitSize(); long totalFileSize = 0; boolean exceedInitialThreshold = false; - for (HiveMetaStoreCache.FileCacheValue fileCacheValue : fileCaches) { + for (HiveExternalMetaCache.FileCacheValue fileCacheValue : fileCaches) { if (fileCacheValue.getFiles() == null) { continue; } - for (HiveMetaStoreCache.HiveFileStatus status : fileCacheValue.getFiles()) { + for (HiveExternalMetaCache.HiveFileStatus status : fileCacheValue.getFiles()) { totalFileSize += status.getLength(); if (!exceedInitialThreshold && totalFileSize >= sessionVariable.getMaxSplitSize() @@ -380,9 +380,9 @@ private long determineTargetFileSplitSize(List fileCaches, } private void splitAllFiles(List allFiles, - List hiveFileStatuses, + List hiveFileStatuses, long realFileSplitSize) throws IOException { - for (HiveMetaStoreCache.HiveFileStatus status : hiveFileStatuses) { + for (HiveExternalMetaCache.HiveFileStatus status : hiveFileStatuses) { allFiles.addAll(fileSplitter.splitFile( status.getPath(), realFileSplitSize, @@ -395,11 +395,11 @@ private void splitAllFiles(List allFiles, } } - private List selectFiles(List inputCacheValue) { - List fileList = Lists.newArrayList(); + private List selectFiles(List inputCacheValue) { + List fileList = Lists.newArrayList(); long totalSize = 0; for (FileCacheValue value : inputCacheValue) { - for (HiveMetaStoreCache.HiveFileStatus file : value.getFiles()) { + for (HiveExternalMetaCache.HiveFileStatus file : value.getFiles()) { file.setSplittable(value.isSplittable()); file.setPartitionValues(value.getPartitionValues()); file.setAcidInfo(value.getAcidInfo()); @@ -420,7 +420,7 @@ private List selectFiles(List long selectedSize = 0; Collections.shuffle(fileList, new Random(tableSample.getSeek())); int index = 0; - for (HiveMetaStoreCache.HiveFileStatus file : fileList) { + for (HiveExternalMetaCache.HiveFileStatus file : fileList) { selectedSize += file.getLength(); index += 1; if (selectedSize >= sampleSize) { @@ -430,7 +430,7 @@ private List selectFiles(List return fileList.subList(0, index); } - private List getFileSplitByTransaction(HiveMetaStoreCache cache, List partitions, + private List getFileSplitByTransaction(HiveExternalMetaCache cache, List partitions, String bindBrokerName) { for (HivePartition partition : partitions) { if (partition.getPartitionValues() == null || partition.getPartitionValues().isEmpty()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiExternalMetaCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiExternalMetaCache.java new file mode 100644 index 00000000000000..74d2aa99900340 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiExternalMetaCache.java @@ -0,0 +1,238 @@ +// 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.datasource.hudi; + +import org.apache.doris.common.util.Util; +import org.apache.doris.datasource.CacheException; +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.doris.datasource.TablePartitionValues; +import org.apache.doris.datasource.hive.HMSExternalCatalog; +import org.apache.doris.datasource.hive.HMSExternalTable; +import org.apache.doris.datasource.hive.HiveMetaStoreClientHelper; +import org.apache.doris.datasource.metacache.AbstractExternalMetaCache; +import org.apache.doris.datasource.metacache.MetaCacheEntryDef; +import org.apache.doris.datasource.metacache.MetaCacheEntryInvalidation; + +import org.apache.hadoop.hive.common.FileUtils; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.FileSystemViewManager; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.stream.Collectors; + +/** + * Hudi engine implementation of {@link AbstractExternalMetaCache}. + * + *

Registered entries: + *

    + *
  • {@code partition}: partition metadata keyed by table identity + snapshot timestamp + mode
  • + *
  • {@code fs_view}: {@link HoodieTableFileSystemView} keyed by {@link NameMapping}
  • + *
  • {@code meta_client}: {@link HoodieTableMetaClient} keyed by {@link NameMapping}
  • + *
  • {@code schema}: Hudi schema cache keyed by table identity + timestamp
  • + *
+ * + *

Invalidation behavior: + *

    + *
  • db/table invalidation clears all four entries for matching keys
  • + *
  • partition-level invalidation currently falls back to table-level invalidation
  • + *
+ */ +public class HudiExternalMetaCache extends AbstractExternalMetaCache { + private static final Logger LOG = LogManager.getLogger(HudiExternalMetaCache.class); + + public static final String ENGINE = "hudi"; + public static final String ENTRY_PARTITION = "partition"; + public static final String ENTRY_FS_VIEW = "fs_view"; + public static final String ENTRY_META_CLIENT = "meta_client"; + public static final String ENTRY_SCHEMA = "schema"; + + private final EntryHandle partitionEntry; + private final EntryHandle fsViewEntry; + private final EntryHandle metaClientEntry; + private final EntryHandle schemaEntry; + + public HudiExternalMetaCache(ExecutorService refreshExecutor) { + super(ENGINE, refreshExecutor); + partitionEntry = registerEntry(MetaCacheEntryDef.of(ENTRY_PARTITION, HudiPartitionCacheKey.class, + TablePartitionValues.class, this::loadPartitionValuesCacheValue, defaultEntryCacheSpec(), + MetaCacheEntryInvalidation.forNameMapping(HudiPartitionCacheKey::getNameMapping))); + fsViewEntry = registerEntry(MetaCacheEntryDef.of(ENTRY_FS_VIEW, HudiFsViewCacheKey.class, + HoodieTableFileSystemView.class, this::createFsView, defaultEntryCacheSpec(), + MetaCacheEntryInvalidation.forNameMapping(HudiFsViewCacheKey::getNameMapping))); + metaClientEntry = registerEntry(MetaCacheEntryDef.of(ENTRY_META_CLIENT, HudiMetaClientCacheKey.class, + HoodieTableMetaClient.class, this::createHoodieTableMetaClient, defaultEntryCacheSpec(), + MetaCacheEntryInvalidation.forNameMapping(HudiMetaClientCacheKey::getNameMapping))); + schemaEntry = registerEntry(MetaCacheEntryDef.of(ENTRY_SCHEMA, HudiSchemaCacheKey.class, + SchemaCacheValue.class, this::loadSchemaCacheValue, defaultSchemaCacheSpec(), + MetaCacheEntryInvalidation.forNameMapping(HudiSchemaCacheKey::getNameMapping))); + } + + public HoodieTableMetaClient getHoodieTableMetaClient(NameMapping nameMapping) { + return metaClientEntry.get(nameMapping.getCtlId()).get(HudiMetaClientCacheKey.of(nameMapping)); + } + + public HoodieTableFileSystemView getFsView(NameMapping nameMapping) { + return fsViewEntry.get(nameMapping.getCtlId()).get(HudiFsViewCacheKey.of(nameMapping)); + } + + public HudiSchemaCacheValue getHudiSchemaCacheValue(NameMapping nameMapping, long timestamp) { + SchemaCacheValue schemaCacheValue = schemaEntry.get(nameMapping.getCtlId()) + .get(new HudiSchemaCacheKey(nameMapping, timestamp)); + return (HudiSchemaCacheValue) schemaCacheValue; + } + + public TablePartitionValues getSnapshotPartitionValues(HMSExternalTable table, + String timestamp, boolean useHiveSyncPartition) { + return partitionEntry.get(table.getCatalog().getId()).get( + HudiPartitionCacheKey.of(table.getOrBuildNameMapping(), Long.parseLong(timestamp), + useHiveSyncPartition)); + } + + public TablePartitionValues getPartitionValues(HMSExternalTable table, boolean useHiveSyncPartition) + throws CacheException { + HoodieTableMetaClient tableMetaClient = getHoodieTableMetaClient(table.getOrBuildNameMapping()); + TablePartitionValues emptyPartitionValues = new TablePartitionValues(); + Option partitionColumns = tableMetaClient.getTableConfig().getPartitionFields(); + if (!partitionColumns.isPresent() || partitionColumns.get().length == 0) { + return emptyPartitionValues; + } + HoodieTimeline timeline = tableMetaClient.getCommitsAndCompactionTimeline().filterCompletedInstants(); + Option lastInstant = timeline.lastInstant(); + if (!lastInstant.isPresent()) { + return emptyPartitionValues; + } + long lastTimestamp = Long.parseLong(lastInstant.get().requestedTime()); + return partitionEntry.get(table.getCatalog().getId()).get( + HudiPartitionCacheKey.of(table.getOrBuildNameMapping(), lastTimestamp, useHiveSyncPartition)); + } + + private HoodieTableFileSystemView createFsView(HudiFsViewCacheKey key) { + HoodieTableMetaClient tableMetaClient = metaClientEntry.get(key.getNameMapping().getCtlId()) + .get(HudiMetaClientCacheKey.of(key.getNameMapping())); + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().build(); + HoodieLocalEngineContext ctx = new HoodieLocalEngineContext(tableMetaClient.getStorageConf()); + return FileSystemViewManager.createInMemoryFileSystemView(ctx, tableMetaClient, metadataConfig); + } + + private HoodieTableMetaClient createHoodieTableMetaClient(HudiMetaClientCacheKey key) { + LOG.debug("create hudi table meta client for {}", key.getNameMapping().getFullLocalName()); + HMSExternalTable hudiTable = findHudiTable(key.getNameMapping()); + HadoopStorageConfiguration hadoopStorageConfiguration = + new HadoopStorageConfiguration(hudiTable.getCatalog().getConfiguration()); + return HiveMetaStoreClientHelper.ugiDoAs( + hudiTable.getCatalog().getConfiguration(), + () -> HoodieTableMetaClient.builder() + .setConf(hadoopStorageConfiguration) + .setBasePath(hudiTable.getRemoteTable().getSd().getLocation()) + .build()); + } + + private TablePartitionValues loadPartitionValuesCacheValue(HudiPartitionCacheKey key) { + HMSExternalTable hudiTable = findHudiTable(key.getNameMapping()); + HoodieTableMetaClient tableMetaClient = getHoodieTableMetaClient(key.getNameMapping()); + return loadPartitionValues(hudiTable, tableMetaClient, key.getTimestamp(), key.isUseHiveSyncPartition()); + } + + private TablePartitionValues loadPartitionValues(HMSExternalTable table, HoodieTableMetaClient tableMetaClient, + long timestamp, boolean useHiveSyncPartition) { + try { + TablePartitionValues partitionValues = new TablePartitionValues(); + Option partitionColumns = tableMetaClient.getTableConfig().getPartitionFields(); + if (!partitionColumns.isPresent() || partitionColumns.get().length == 0) { + return partitionValues; + } + HoodieTimeline timeline = tableMetaClient.getCommitsAndCompactionTimeline().filterCompletedInstants(); + List partitionNames = loadPartitionNames(table, tableMetaClient, timeline, timestamp, + useHiveSyncPartition); + List partitionColumnsList = Arrays.asList(partitionColumns.get()); + partitionValues.addPartitions(partitionNames, + partitionNames.stream() + .map(partition -> HudiPartitionUtils.parsePartitionValues(partitionColumnsList, partition)) + .collect(Collectors.toList()), + table.getHudiPartitionColumnTypes(timestamp), + Collections.nCopies(partitionNames.size(), 0L)); + partitionValues.setLastUpdateTimestamp(timestamp); + return partitionValues; + } catch (Exception e) { + LOG.warn("Failed to get hudi partitions", e); + throw new CacheException("Failed to get hudi partitions: " + Util.getRootCauseMessage(e), e); + } + } + + private List loadPartitionNames(HMSExternalTable table, HoodieTableMetaClient tableMetaClient, + HoodieTimeline timeline, long timestamp, boolean useHiveSyncPartition) throws Exception { + Option lastInstant = timeline.lastInstant(); + if (!lastInstant.isPresent()) { + return Collections.emptyList(); + } + long lastTimestamp = Long.parseLong(lastInstant.get().requestedTime()); + if (timestamp != lastTimestamp) { + return HudiPartitionUtils.getPartitionNamesBeforeOrEquals(timeline, String.valueOf(timestamp)); + } + if (!useHiveSyncPartition) { + return HudiPartitionUtils.getAllPartitionNames(tableMetaClient); + } + HMSExternalCatalog catalog = (HMSExternalCatalog) table.getCatalog(); + List partitionNames = catalog.getClient() + .listPartitionNames(table.getRemoteDbName(), table.getRemoteName()); + partitionNames = partitionNames.stream().map(FileUtils::unescapePathName).collect(Collectors.toList()); + if (partitionNames.isEmpty()) { + LOG.warn("Failed to get partitions from hms api, switch it from hudi api."); + return HudiPartitionUtils.getAllPartitionNames(tableMetaClient); + } + return partitionNames; + } + + private HMSExternalTable findHudiTable(NameMapping nameMapping) { + ExternalTable dorisTable = findExternalTable(nameMapping, ENGINE); + if (!(dorisTable instanceof HMSExternalTable)) { + throw new CacheException("table %s.%s.%s is not hms external table when loading hudi cache", + null, nameMapping.getCtlId(), nameMapping.getLocalDbName(), nameMapping.getLocalTblName()); + } + return (HMSExternalTable) dorisTable; + } + + private SchemaCacheValue loadSchemaCacheValue(HudiSchemaCacheKey key) { + ExternalTable dorisTable = findExternalTable(key.getNameMapping(), ENGINE); + return dorisTable.initSchemaAndUpdateTime(key).orElseThrow(() -> + new CacheException("failed to load hudi schema cache value for: %s.%s.%s, timestamp: %s", + null, key.getNameMapping().getCtlId(), key.getNameMapping().getLocalDbName(), + key.getNameMapping().getLocalTblName(), key.getTimestamp())); + } + + @Override + protected Map catalogPropertyCompatibilityMap() { + return singleCompatibilityMap(ExternalCatalog.SCHEMA_CACHE_TTL_SECOND, ENTRY_SCHEMA); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiFsViewCacheKey.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiFsViewCacheKey.java new file mode 100644 index 00000000000000..385cb6878893eb --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiFsViewCacheKey.java @@ -0,0 +1,58 @@ +// 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.datasource.hudi; + +import org.apache.doris.datasource.NameMapping; + +import java.util.Objects; + +/** + * Cache key for hudi fs view entry. + */ +public final class HudiFsViewCacheKey { + private final NameMapping nameMapping; + + private HudiFsViewCacheKey(NameMapping nameMapping) { + this.nameMapping = Objects.requireNonNull(nameMapping, "nameMapping can not be null"); + } + + public static HudiFsViewCacheKey of(NameMapping nameMapping) { + return new HudiFsViewCacheKey(nameMapping); + } + + public NameMapping getNameMapping() { + return nameMapping; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + HudiFsViewCacheKey that = (HudiFsViewCacheKey) o; + return Objects.equals(nameMapping, that.nameMapping); + } + + @Override + public int hashCode() { + return Objects.hash(nameMapping); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiMetaClientCacheKey.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiMetaClientCacheKey.java new file mode 100644 index 00000000000000..2f2ba0e032da9e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiMetaClientCacheKey.java @@ -0,0 +1,58 @@ +// 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.datasource.hudi; + +import org.apache.doris.datasource.NameMapping; + +import java.util.Objects; + +/** + * Cache key for hudi meta client entry. + */ +public final class HudiMetaClientCacheKey { + private final NameMapping nameMapping; + + private HudiMetaClientCacheKey(NameMapping nameMapping) { + this.nameMapping = Objects.requireNonNull(nameMapping, "nameMapping can not be null"); + } + + public static HudiMetaClientCacheKey of(NameMapping nameMapping) { + return new HudiMetaClientCacheKey(nameMapping); + } + + public NameMapping getNameMapping() { + return nameMapping; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + HudiMetaClientCacheKey that = (HudiMetaClientCacheKey) o; + return nameMapping.equals(that.nameMapping); + } + + @Override + public int hashCode() { + return Objects.hash(nameMapping); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiPartitionCacheKey.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiPartitionCacheKey.java new file mode 100644 index 00000000000000..b39688acaf5b47 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiPartitionCacheKey.java @@ -0,0 +1,72 @@ +// 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.datasource.hudi; + +import org.apache.doris.datasource.NameMapping; + +import java.util.Objects; + +/** + * Cache key for Hudi partition metadata by table, snapshot timestamp and load mode. + */ +public final class HudiPartitionCacheKey { + private final NameMapping nameMapping; + private final long timestamp; + private final boolean useHiveSyncPartition; + + private HudiPartitionCacheKey(NameMapping nameMapping, long timestamp, boolean useHiveSyncPartition) { + this.nameMapping = Objects.requireNonNull(nameMapping, "nameMapping can not be null"); + this.timestamp = timestamp; + this.useHiveSyncPartition = useHiveSyncPartition; + } + + public static HudiPartitionCacheKey of(NameMapping nameMapping, long timestamp, boolean useHiveSyncPartition) { + return new HudiPartitionCacheKey(nameMapping, timestamp, useHiveSyncPartition); + } + + public NameMapping getNameMapping() { + return nameMapping; + } + + public long getTimestamp() { + return timestamp; + } + + public boolean isUseHiveSyncPartition() { + return useHiveSyncPartition; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof HudiPartitionCacheKey)) { + return false; + } + HudiPartitionCacheKey that = (HudiPartitionCacheKey) o; + return timestamp == that.timestamp + && useHiveSyncPartition == that.useHiveSyncPartition + && Objects.equals(nameMapping, that.nameMapping); + } + + @Override + public int hashCode() { + return Objects.hash(nameMapping, timestamp, useHiveSyncPartition); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiPartitionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiPartitionUtils.java new file mode 100644 index 00000000000000..4a22f7eae8d165 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiPartitionUtils.java @@ -0,0 +1,90 @@ +// 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.datasource.hudi; + +import org.apache.hadoop.hive.common.FileUtils; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +/** + * Hudi partition utility helpers shared by scan/planner/meta cache components. + */ +public final class HudiPartitionUtils { + private HudiPartitionUtils() { + } + + public static List getAllPartitionNames(HoodieTableMetaClient tableMetaClient) throws IOException { + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() + .enable(HoodieTableMetadataUtil.isFilesPartitionAvailable(tableMetaClient)) + .build(); + HoodieTableMetadata newTableMetadata = HoodieTableMetadata.create( + new HoodieLocalEngineContext(tableMetaClient.getStorageConf()), tableMetaClient.getStorage(), + metadataConfig, tableMetaClient.getBasePath().toString(), true); + return newTableMetadata.getAllPartitionPaths(); + } + + public static List getPartitionNamesBeforeOrEquals(HoodieTimeline timeline, String timestamp) { + return new ArrayList<>(HoodieTableMetadataUtil.getWritePartitionPaths( + timeline.findInstantsBeforeOrEquals(timestamp).getInstants().stream().map(instant -> { + try { + return TimelineUtils.getCommitMetadata(instant, timeline); + } catch (IOException e) { + throw new RuntimeException(e.getMessage(), e); + } + }).collect(Collectors.toList()))); + } + + public static List parsePartitionValues(List partitionColumns, String partitionPath) { + if (partitionColumns.size() == 0) { + // This is a non-partitioned table. + return Collections.emptyList(); + } + String[] partitionFragments = partitionPath.split("/"); + if (partitionFragments.length != partitionColumns.size()) { + if (partitionColumns.size() == 1) { + // If partition column size is 1, map whole partition path to this single partition column. + String prefix = partitionColumns.get(0) + "="; + String partitionValue = partitionPath.startsWith(prefix) + ? partitionPath.substring(prefix.length()) : partitionPath; + return Collections.singletonList(FileUtils.unescapePathName(partitionValue)); + } + throw new RuntimeException("Failed to parse partition values of path: " + partitionPath); + } + List partitionValues = new ArrayList<>(partitionFragments.length); + for (int i = 0; i < partitionFragments.length; i++) { + String prefix = partitionColumns.get(i) + "="; + if (partitionFragments[i].startsWith(prefix)) { + partitionValues.add(FileUtils.unescapePathName(partitionFragments[i].substring(prefix.length()))); + } else { + partitionValues.add(FileUtils.unescapePathName(partitionFragments[i])); + } + } + return partitionValues; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiSchemaCacheKey.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiSchemaCacheKey.java index 2b0d125494533b..3ab8409858de14 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiSchemaCacheKey.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiSchemaCacheKey.java @@ -17,8 +17,8 @@ package org.apache.doris.datasource.hudi; -import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey; import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.SchemaCacheKey; import com.google.common.base.Objects; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java index a7f9c3d09bdc10..fbfe7a0a4daf3e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java @@ -26,14 +26,11 @@ import org.apache.doris.catalog.StructField; import org.apache.doris.catalog.StructType; import org.apache.doris.catalog.Type; -import org.apache.doris.datasource.ExternalSchemaCache; -import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey; import org.apache.doris.datasource.SchemaCacheValue; import org.apache.doris.datasource.TablePartitionValues; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.hive.HiveMetaStoreClientHelper; import org.apache.doris.datasource.hive.HivePartition; -import org.apache.doris.datasource.hudi.source.HudiCachedPartitionProcessor; import org.apache.doris.thrift.TColumnType; import org.apache.doris.thrift.TPrimitiveType; import org.apache.doris.thrift.schema.external.TArrayField; @@ -285,8 +282,9 @@ public static TablePartitionValues getPartitionValues(Optional ta TablePartitionValues partitionValues = new TablePartitionValues(); HoodieTableMetaClient hudiClient = hmsTable.getHudiClient(); - HudiCachedPartitionProcessor processor = (HudiCachedPartitionProcessor) Env.getCurrentEnv() - .getExtMetaCacheMgr().getHudiPartitionProcess(hmsTable.getCatalog()); + HudiExternalMetaCache hudiExternalMetaCache = + Env.getCurrentEnv().getExtMetaCacheMgr() + .hudi(hmsTable.getCatalog().getId()); boolean useHiveSyncPartition = hmsTable.useHiveSyncPartition(); if (tableSnapshot.isPresent()) { @@ -297,7 +295,7 @@ public static TablePartitionValues getPartitionValues(Optional ta String queryInstant = tableSnapshot.get().getValue().replaceAll("[-: ]", ""); try { partitionValues = hmsTable.getCatalog().getExecutionAuthenticator().execute(() -> - processor.getSnapshotPartitionValues(hmsTable, hudiClient, queryInstant, useHiveSyncPartition)); + hudiExternalMetaCache.getSnapshotPartitionValues(hmsTable, queryInstant, useHiveSyncPartition)); } catch (Exception e) { throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e), e); } @@ -309,7 +307,7 @@ public static TablePartitionValues getPartitionValues(Optional ta } try { partitionValues = hmsTable.getCatalog().getExecutionAuthenticator().execute(() - -> processor.getPartitionValues(hmsTable, hudiClient, useHiveSyncPartition)); + -> hudiExternalMetaCache.getPartitionValues(hmsTable, useHiveSyncPartition)); } catch (Exception e) { throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e), e); } @@ -328,9 +326,9 @@ public static HoodieTableMetaClient buildHudiTableMetaClient(String hudiBasePath public static HudiSchemaCacheValue getSchemaCacheValue(HMSExternalTable hmsTable, String queryInstant) { - ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(hmsTable.getCatalog()); - SchemaCacheKey key = new HudiSchemaCacheKey(hmsTable.getOrBuildNameMapping(), Long.parseLong(queryInstant)); - Optional schemaCacheValue = cache.getSchemaValue(key); + Optional schemaCacheValue = Env.getCurrentEnv().getExtMetaCacheMgr() + .getSchemaCacheValue(hmsTable, + new HudiSchemaCacheKey(hmsTable.getOrBuildNameMapping(), Long.parseLong(queryInstant))); return (HudiSchemaCacheValue) schemaCacheValue.get(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java index b5989cc786da56..fbe9cd4e417a8c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java @@ -19,6 +19,7 @@ import org.apache.doris.common.util.LocationPath; import org.apache.doris.datasource.TableFormatType; +import org.apache.doris.datasource.hudi.HudiPartitionUtils; import org.apache.doris.spi.Split; import org.apache.hadoop.conf.Configuration; @@ -217,7 +218,7 @@ public List collectSplits() throws HoodieException { LocationPath locationPath = LocationPath.of(baseFile); HudiSplit hudiSplit = new HudiSplit(locationPath, 0, stat.getFileSizeInBytes(), stat.getFileSizeInBytes(), new String[0], - HudiPartitionProcessor.parsePartitionValues(partitionNames, stat.getPartitionPath())); + HudiPartitionUtils.parsePartitionValues(partitionNames, stat.getPartitionPath())); hudiSplit.setTableFormatType(TableFormatType.HUDI); splits.add(hudiSplit); }; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedFsViewProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedFsViewProcessor.java deleted file mode 100644 index 84dccb38574031..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedFsViewProcessor.java +++ /dev/null @@ -1,136 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.datasource.hudi.source; - -import org.apache.doris.common.CacheFactory; -import org.apache.doris.common.Config; -import org.apache.doris.datasource.ExternalMetaCacheMgr; -import org.apache.doris.datasource.ExternalTable; - -import com.github.benmanes.caffeine.cache.LoadingCache; -import com.google.common.collect.Maps; -import org.apache.hudi.common.config.HoodieMetadataConfig; -import org.apache.hudi.common.engine.HoodieLocalEngineContext; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.view.FileSystemViewManager; -import org.apache.hudi.common.table.view.HoodieTableFileSystemView; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.Map; -import java.util.Objects; -import java.util.OptionalLong; -import java.util.concurrent.ExecutorService; - -public class HudiCachedFsViewProcessor { - private static final Logger LOG = LogManager.getLogger(HudiCachedFsViewProcessor.class); - private final LoadingCache fsViewCache; - - public HudiCachedFsViewProcessor(ExecutorService executor) { - CacheFactory partitionCacheFactory = new CacheFactory( - OptionalLong.of(Config.external_cache_expire_time_seconds_after_access), - OptionalLong.of(Config.external_cache_refresh_time_minutes * 60), - Config.max_external_table_cache_num, - true, - null); - this.fsViewCache = partitionCacheFactory.buildCache(this::createFsView, executor); - } - - private HoodieTableFileSystemView createFsView(FsViewKey key) { - HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().build(); - HoodieLocalEngineContext ctx = new HoodieLocalEngineContext(key.getClient().getStorageConf()); - return FileSystemViewManager.createInMemoryFileSystemView(ctx, key.getClient(), metadataConfig); - } - - public HoodieTableFileSystemView getFsView(String dbName, String tbName, HoodieTableMetaClient hudiClient) { - return fsViewCache.get(new FsViewKey(dbName, tbName, hudiClient)); - } - - public void cleanUp() { - fsViewCache.cleanUp(); - } - - public void invalidateAll() { - fsViewCache.invalidateAll(); - } - - public void invalidateDbCache(String dbName) { - fsViewCache.asMap().forEach((k, v) -> { - if (k.getDbName().equals(dbName)) { - fsViewCache.invalidate(k); - } - }); - } - - public void invalidateTableCache(ExternalTable dorisTable) { - fsViewCache.asMap().forEach((k, v) -> { - if (k.getDbName().equals(dorisTable.getDbName()) && k.getTbName().equals(dorisTable.getName())) { - fsViewCache.invalidate(k); - } - }); - } - - private static class FsViewKey { - String dbName; - String tbName; - HoodieTableMetaClient client; - - public FsViewKey(String dbName, String tbName, HoodieTableMetaClient client) { - this.dbName = dbName; - this.tbName = tbName; - this.client = client; - } - - public String getDbName() { - return dbName; - } - - public String getTbName() { - return tbName; - } - - public HoodieTableMetaClient getClient() { - return client; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - FsViewKey fsViewKey = (FsViewKey) o; - return Objects.equals(dbName, fsViewKey.dbName) && Objects.equals(tbName, fsViewKey.tbName) - && Objects.equals(client.getBasePath(), fsViewKey.client.getBasePath()); - } - - @Override - public int hashCode() { - return Objects.hash(dbName, tbName, client.getBasePath()); - } - } - - public Map> getCacheStats() { - Map> res = Maps.newHashMap(); - res.put("hudi_fs_view_cache", - ExternalMetaCacheMgr.getCacheStats(fsViewCache.stats(), fsViewCache.estimatedSize())); - return res; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedMetaClientProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedMetaClientProcessor.java deleted file mode 100644 index bb9351d41219d1..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedMetaClientProcessor.java +++ /dev/null @@ -1,147 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.datasource.hudi.source; - -import org.apache.doris.common.CacheFactory; -import org.apache.doris.common.Config; -import org.apache.doris.datasource.ExternalMetaCacheMgr; -import org.apache.doris.datasource.ExternalTable; -import org.apache.doris.datasource.NameMapping; -import org.apache.doris.datasource.hive.HiveMetaStoreClientHelper; - -import com.github.benmanes.caffeine.cache.LoadingCache; -import com.google.common.collect.Maps; -import org.apache.hadoop.conf.Configuration; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.Map; -import java.util.Objects; -import java.util.OptionalLong; -import java.util.concurrent.ExecutorService; - -public class HudiCachedMetaClientProcessor { - private static final Logger LOG = LogManager.getLogger(HudiCachedMetaClientProcessor.class); - private final LoadingCache hudiTableMetaClientCache; - - public HudiCachedMetaClientProcessor(ExecutorService executor) { - CacheFactory partitionCacheFactory = new CacheFactory( - OptionalLong.of(Config.external_cache_expire_time_seconds_after_access), - OptionalLong.of(Config.external_cache_refresh_time_minutes * 60), - Config.max_external_table_cache_num, - true, - null); - - this.hudiTableMetaClientCache = partitionCacheFactory.buildCache( - this::createHoodieTableMetaClient, - executor); - } - - private HoodieTableMetaClient createHoodieTableMetaClient(HudiCachedClientKey key) { - LOG.debug("create hudi table meta client for {}.{}", key.getNameMapping().getFullLocalName()); - HadoopStorageConfiguration hadoopStorageConfiguration = new HadoopStorageConfiguration(key.getConf()); - return HiveMetaStoreClientHelper.ugiDoAs( - key.getConf(), - () -> HoodieTableMetaClient - .builder() - .setConf(hadoopStorageConfiguration) - .setBasePath(key.getHudiBasePath()) - .build()); - } - - public HoodieTableMetaClient getHoodieTableMetaClient( - NameMapping nameMapping, String hudiBasePath, Configuration conf) { - return hudiTableMetaClientCache.get(new HudiCachedClientKey(nameMapping, hudiBasePath, conf)); - } - - public void cleanUp() { - hudiTableMetaClientCache.cleanUp(); - } - - public void invalidateAll() { - hudiTableMetaClientCache.invalidateAll(); - } - - public void invalidateDbCache(String dbName) { - hudiTableMetaClientCache.asMap().forEach((k, v) -> { - if (k.getNameMapping().getLocalDbName().equals(dbName)) { - hudiTableMetaClientCache.invalidate(k); - } - }); - } - - public void invalidateTableCache(ExternalTable dorisTable) { - hudiTableMetaClientCache.asMap().forEach((k, v) -> { - if (k.getNameMapping().getLocalDbName().equals(dorisTable.getDbName()) - && k.getNameMapping().getLocalTblName().equals(dorisTable.getName())) { - hudiTableMetaClientCache.invalidate(k); - } - }); - } - - private static class HudiCachedClientKey { - NameMapping nameMapping; - String hudiBasePath; - Configuration conf; - - public HudiCachedClientKey(NameMapping nameMapping, String hudiBasePath, Configuration conf) { - this.nameMapping = nameMapping; - this.hudiBasePath = hudiBasePath; - this.conf = conf; - } - - public NameMapping getNameMapping() { - return nameMapping; - } - - public String getHudiBasePath() { - return hudiBasePath; - } - - public Configuration getConf() { - return conf; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - HudiCachedClientKey that = (HudiCachedClientKey) o; - return nameMapping.equals(that.nameMapping) - && Objects.equals(hudiBasePath, that.hudiBasePath); - } - - @Override - public int hashCode() { - return Objects.hash(nameMapping, hudiBasePath); - } - } - - public Map> getCacheStats() { - Map> res = Maps.newHashMap(); - res.put("hudi_meta_client_cache", ExternalMetaCacheMgr.getCacheStats(hudiTableMetaClientCache.stats(), - hudiTableMetaClientCache.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 deleted file mode 100644 index 7a2824e7b063d9..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedPartitionProcessor.java +++ /dev/null @@ -1,193 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.datasource.hudi.source; - -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.ExternalTable; -import org.apache.doris.datasource.TablePartitionValues; -import org.apache.doris.datasource.TablePartitionValues.TablePartitionKey; -import org.apache.doris.datasource.hive.HMSExternalCatalog; -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.hadoop.hive.common.FileUtils; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.Option; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.OptionalLong; -import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; -import java.util.stream.Collectors; - -public class HudiCachedPartitionProcessor extends HudiPartitionProcessor { - private static final Logger LOG = LoggerFactory.getLogger(HudiCachedPartitionProcessor.class); - private final long catalogId; - private final Executor executor; - private final LoadingCache partitionCache; - - public HudiCachedPartitionProcessor(long catalogId, ExecutorService executor) { - this.catalogId = catalogId; - this.executor = executor; - CacheFactory partitionCacheFactory = new CacheFactory( - OptionalLong.of(Config.external_cache_expire_time_seconds_after_access), - OptionalLong.of(Config.external_cache_refresh_time_minutes * 60), - Config.max_external_table_cache_num, - true, - null); - this.partitionCache = partitionCacheFactory.buildCache(key -> new TablePartitionValues(), executor); - } - - @Override - public void cleanUp() { - partitionCache.cleanUp(); - } - - @Override - public void cleanDatabasePartitions(String dbName) { - partitionCache.asMap().keySet().stream().filter(k -> k.getDbName().equals(dbName)).collect(Collectors.toList()) - .forEach(partitionCache::invalidate); - } - - @Override - public void cleanTablePartitions(ExternalTable dorisTable) { - partitionCache.asMap().keySet().stream() - .filter(k -> k.getDbName().equals(dorisTable.getDbName()) - && k.getTblName().equals(dorisTable.getName())) - .collect(Collectors.toList()) - .forEach(partitionCache::invalidate); - } - - public TablePartitionValues getSnapshotPartitionValues(HMSExternalTable table, - HoodieTableMetaClient tableMetaClient, String timestamp, boolean useHiveSyncPartition) { - Preconditions.checkState(catalogId == table.getCatalog().getId()); - TablePartitionValues partitionValues = new TablePartitionValues(); - Option partitionColumns = tableMetaClient.getTableConfig().getPartitionFields(); - if (!partitionColumns.isPresent() || partitionColumns.get().length == 0) { - return partitionValues; - } - HoodieTimeline timeline = tableMetaClient.getCommitsAndCompactionTimeline().filterCompletedInstants(); - Option lastInstant = timeline.lastInstant(); - if (!lastInstant.isPresent()) { - return partitionValues; - } - long lastTimestamp = Long.parseLong(lastInstant.get().requestedTime()); - if (Long.parseLong(timestamp) == lastTimestamp) { - return getPartitionValues(table, tableMetaClient, useHiveSyncPartition); - } - List partitionNameAndValues = getPartitionNamesBeforeOrEquals(timeline, timestamp); - List partitionNames = Arrays.asList(partitionColumns.get()); - // we don't support auto refresh hudi mtmv currently, - // so the list `partitionLastUpdateTimestamp` is full of 0L. - partitionValues.addPartitions(partitionNameAndValues, - partitionNameAndValues.stream().map(p -> parsePartitionValues(partitionNames, p)) - .collect(Collectors.toList()), table.getHudiPartitionColumnTypes(Long.parseLong(timestamp)), - Collections.nCopies(partitionNameAndValues.size(), 0L)); - partitionValues.setLastUpdateTimestamp(Long.parseLong(timestamp)); - return partitionValues; - } - - public TablePartitionValues getPartitionValues(HMSExternalTable table, HoodieTableMetaClient tableMetaClient, - boolean useHiveSyncPartition) - throws CacheException { - Preconditions.checkState(catalogId == table.getCatalog().getId()); - TablePartitionValues partitionValues = new TablePartitionValues(); - Option partitionColumns = tableMetaClient.getTableConfig().getPartitionFields(); - if (!partitionColumns.isPresent() || partitionColumns.get().length == 0) { - return partitionValues; - } - HoodieTimeline timeline = tableMetaClient.getCommitsAndCompactionTimeline().filterCompletedInstants(); - Option lastInstant = timeline.lastInstant(); - if (!lastInstant.isPresent()) { - return partitionValues; - } - try { - long lastTimestamp = Long.parseLong(lastInstant.get().requestedTime()); - partitionValues = partitionCache.get( - new TablePartitionKey(table.getDbName(), table.getName(), - table.getHudiPartitionColumnTypes(lastTimestamp))); - partitionValues.readLock().lock(); - try { - long lastUpdateTimestamp = partitionValues.getLastUpdateTimestamp(); - if (lastTimestamp <= lastUpdateTimestamp) { - return partitionValues; - } - } finally { - partitionValues.readLock().unlock(); - } - - partitionValues.writeLock().lock(); - try { - HMSExternalCatalog catalog = (HMSExternalCatalog) table.getCatalog(); - List partitionNames; - if (useHiveSyncPartition) { - // When a Hudi table is synchronized to HMS, the partition information is also synchronized, - // so even if the metastore is not enabled in the Hudi table - // (for example, if the Metastore is false for a Hudi table created with Flink), - // we can still obtain the partition information through the HMS API. - partitionNames = catalog.getClient() - .listPartitionNames(table.getRemoteDbName(), table.getRemoteName()); - // HMS stored Hudi partition paths may have double encoding issue (e.g., %3A - // becomes %253A), need to unescape first here. - partitionNames = partitionNames.stream() - .map(FileUtils::unescapePathName) - .collect(Collectors.toList()); - if (partitionNames.size() == 0) { - LOG.warn("Failed to get partitions from hms api, switch it from hudi api."); - partitionNames = getAllPartitionNames(tableMetaClient); - } - } else { - partitionNames = getAllPartitionNames(tableMetaClient); - } - List partitionColumnsList = Arrays.asList(partitionColumns.get()); - partitionValues.cleanPartitions(); - partitionValues.addPartitions(partitionNames, - partitionNames.stream().map(p -> parsePartitionValues(partitionColumnsList, p)) - .collect(Collectors.toList()), table.getHudiPartitionColumnTypes(lastTimestamp), - Collections.nCopies(partitionNames.size(), 0L)); - partitionValues.setLastUpdateTimestamp(lastTimestamp); - return partitionValues; - } finally { - partitionValues.writeLock().unlock(); - } - } catch (Exception e) { - LOG.warn("Failed to get hudi partitions", e); - throw new CacheException("Failed to get hudi partitions: " + Util.getRootCauseMessage(e), e); - } - } - - public Map> getCacheStats() { - Map> 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/hudi/source/HudiMetadataCacheMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiMetadataCacheMgr.java deleted file mode 100644 index 4d9e654092d6a5..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiMetadataCacheMgr.java +++ /dev/null @@ -1,145 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.datasource.hudi.source; - -import org.apache.doris.datasource.CatalogIf; -import org.apache.doris.datasource.ExternalTable; -import org.apache.doris.datasource.hive.HMSExternalCatalog; - -import com.google.common.collect.Maps; - -import java.util.Map; -import java.util.concurrent.ExecutorService; - -public class HudiMetadataCacheMgr { - private final Map partitionProcessors = Maps.newConcurrentMap(); - private final Map fsViewProcessors = Maps.newConcurrentMap(); - private final Map metaClientProcessors = Maps.newConcurrentMap(); - - private final ExecutorService executor; - - public HudiMetadataCacheMgr(ExecutorService executor) { - this.executor = executor; - } - - public HudiPartitionProcessor getPartitionProcessor(CatalogIf catalog) { - return partitionProcessors.computeIfAbsent(catalog.getId(), catalogId -> { - if (catalog instanceof HMSExternalCatalog) { - return new HudiCachedPartitionProcessor(catalogId, executor); - } else { - throw new RuntimeException("Hudi only supports hive(or compatible) catalog now"); - } - }); - } - - public HudiCachedFsViewProcessor getFsViewProcessor(CatalogIf catalog) { - return fsViewProcessors.computeIfAbsent(catalog.getId(), catalogId -> { - if (catalog instanceof HMSExternalCatalog) { - return new HudiCachedFsViewProcessor(executor); - } else { - throw new RuntimeException("Hudi only supports hive(or compatible) catalog now"); - } - }); - } - - public HudiCachedMetaClientProcessor getHudiMetaClientProcessor(CatalogIf catalog) { - return metaClientProcessors.computeIfAbsent(catalog.getId(), catalogId -> { - if (catalog instanceof HMSExternalCatalog) { - return new HudiCachedMetaClientProcessor(executor); - } else { - throw new RuntimeException("Hudi only supports hive(or compatible) catalog now"); - } - }); - } - - public void removeCache(long catalogId) { - HudiPartitionProcessor partitionProcessor = partitionProcessors.remove(catalogId); - if (partitionProcessor != null) { - partitionProcessor.cleanUp(); - } - HudiCachedFsViewProcessor fsViewProcessor = fsViewProcessors.remove(catalogId); - if (fsViewProcessor != null) { - fsViewProcessor.cleanUp(); - } - HudiCachedMetaClientProcessor metaClientProcessor = metaClientProcessors.remove(catalogId); - if (metaClientProcessor != null) { - metaClientProcessor.cleanUp(); - } - } - - public void invalidateCatalogCache(long catalogId) { - HudiPartitionProcessor processor = partitionProcessors.get(catalogId); - if (processor != null) { - processor.cleanUp(); - } - HudiCachedFsViewProcessor fsViewProcessor = fsViewProcessors.get(catalogId); - if (fsViewProcessor != null) { - fsViewProcessor.invalidateAll(); - } - HudiCachedMetaClientProcessor metaClientProcessor = metaClientProcessors.get(catalogId); - if (metaClientProcessor != null) { - metaClientProcessor.invalidateAll(); - } - } - - public void invalidateDbCache(long catalogId, String dbName) { - HudiPartitionProcessor processor = partitionProcessors.get(catalogId); - if (processor != null) { - processor.cleanDatabasePartitions(dbName); - } - HudiCachedFsViewProcessor fsViewProcessor = fsViewProcessors.get(catalogId); - if (fsViewProcessor != null) { - fsViewProcessor.invalidateDbCache(dbName); - } - HudiCachedMetaClientProcessor metaClientProcessor = metaClientProcessors.get(catalogId); - if (metaClientProcessor != null) { - metaClientProcessor.invalidateDbCache(dbName); - } - } - - public void invalidateTableCache(ExternalTable dorisTable) { - long catalogId = dorisTable.getCatalog().getId(); - HudiPartitionProcessor processor = partitionProcessors.get(catalogId); - if (processor != null) { - processor.cleanTablePartitions(dorisTable); - } - HudiCachedFsViewProcessor fsViewProcessor = fsViewProcessors.get(catalogId); - if (fsViewProcessor != null) { - fsViewProcessor.invalidateTableCache(dorisTable); - } - HudiCachedMetaClientProcessor metaClientProcessor = metaClientProcessors.get(catalogId); - if (metaClientProcessor != null) { - metaClientProcessor.invalidateTableCache(dorisTable); - } - } - - public Map> getCacheStats(CatalogIf catalog) { - Map> res = Maps.newHashMap(); - - HudiCachedPartitionProcessor partitionProcessor = (HudiCachedPartitionProcessor) getPartitionProcessor(catalog); - res.putAll(partitionProcessor.getCacheStats()); - - HudiCachedFsViewProcessor fsViewProcessor = getFsViewProcessor(catalog); - res.putAll(fsViewProcessor.getCacheStats()); - - HudiCachedMetaClientProcessor metaClientProcessor = getHudiMetaClientProcessor(catalog); - res.putAll(metaClientProcessor.getCacheStats()); - - return res; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiPartitionProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiPartitionProcessor.java deleted file mode 100644 index ae127f9e1c4148..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiPartitionProcessor.java +++ /dev/null @@ -1,132 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.datasource.hudi.source; - -import org.apache.doris.datasource.ExternalTable; - -import org.apache.hadoop.hive.common.FileUtils; -import org.apache.hudi.common.config.HoodieMetadataConfig; -import org.apache.hudi.common.engine.HoodieLocalEngineContext; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.timeline.TimelineUtils; -import org.apache.hudi.metadata.HoodieTableMetadata; -import org.apache.hudi.metadata.HoodieTableMetadataUtil; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; - -public abstract class HudiPartitionProcessor { - - public abstract void cleanUp(); - - public abstract void cleanDatabasePartitions(String dbName); - - public abstract void cleanTablePartitions(ExternalTable dorisTable); - - public String[] getPartitionColumns(HoodieTableMetaClient tableMetaClient) { - return tableMetaClient.getTableConfig().getPartitionFields().get(); - } - - public List getAllPartitionNames(HoodieTableMetaClient tableMetaClient) throws IOException { - HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() - .enable(HoodieTableMetadataUtil.isFilesPartitionAvailable(tableMetaClient)) - .build(); - - HoodieTableMetadata newTableMetadata = HoodieTableMetadata.create( - new HoodieLocalEngineContext(tableMetaClient.getStorageConf()), tableMetaClient.getStorage(), - metadataConfig, - tableMetaClient.getBasePath().toString(), true); - - return newTableMetadata.getAllPartitionPaths(); - } - - public List getPartitionNamesBeforeOrEquals(HoodieTimeline timeline, String timestamp) { - return new ArrayList<>(HoodieTableMetadataUtil.getWritePartitionPaths( - timeline.findInstantsBeforeOrEquals(timestamp).getInstants().stream().map(instant -> { - try { - return TimelineUtils.getCommitMetadata(instant, timeline); - } catch (IOException e) { - throw new RuntimeException(e.getMessage(), e); - } - }).collect(Collectors.toList()))); - } - - public List getPartitionNamesInRange(HoodieTimeline timeline, String startTimestamp, String endTimestamp) { - return new ArrayList<>(HoodieTableMetadataUtil.getWritePartitionPaths( - timeline.findInstantsInRange(startTimestamp, endTimestamp).getInstants().stream().map(instant -> { - try { - return TimelineUtils.getCommitMetadata(instant, timeline); - } catch (IOException e) { - throw new RuntimeException(e.getMessage(), e); - } - }).collect(Collectors.toList()))); - } - - public static List parsePartitionValues(List partitionColumns, String partitionPath) { - if (partitionColumns.size() == 0) { - // This is a non-partitioned table - return Collections.emptyList(); - } - String[] partitionFragments = partitionPath.split("/"); - if (partitionFragments.length != partitionColumns.size()) { - if (partitionColumns.size() == 1) { - // If the partition column size is not equal to the partition fragment size - // and the partition column size is 1, we map the whole partition path - // to the partition column which can benefit from the partition prune. - String prefix = partitionColumns.get(0) + "="; - String partitionValue; - if (partitionPath.startsWith(prefix)) { - // support hive style partition path - partitionValue = partitionPath.substring(prefix.length()); - } else { - partitionValue = partitionPath; - } - // In hive, the specific characters like '=', '/' will be url encoded - return Collections.singletonList(FileUtils.unescapePathName(partitionValue)); - } else { - // If the partition column size is not equal to the partition fragments size - // and the partition column size > 1, we do not know how to map the partition - // fragments to the partition columns and therefore return an empty tuple. We - // don't - // fail outright so that in some cases we can fallback to reading the table as - // non-partitioned - // one - throw new RuntimeException("Failed to parse partition values of path: " + partitionPath); - } - } else { - // If partitionSeqs.length == partitionSchema.fields.length - // Append partition name to the partition value if the - // HIVE_STYLE_PARTITIONING is disable. - // e.g. convert "/xx/xx/2021/02" to "/xx/xx/year=2021/month=02" - List partitionValues = new ArrayList<>(partitionFragments.length); - for (int i = 0; i < partitionFragments.length; i++) { - String prefix = partitionColumns.get(i) + "="; - if (partitionFragments[i].startsWith(prefix)) { - partitionValues.add(FileUtils.unescapePathName(partitionFragments[i].substring(prefix.length()))); - } else { - partitionValues.add(FileUtils.unescapePathName(partitionFragments[i])); - } - } - return partitionValues; - } - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java index 0b63d4d539af43..75014c83ffd552 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java @@ -35,6 +35,7 @@ import org.apache.doris.datasource.TableFormatType; import org.apache.doris.datasource.hive.HivePartition; import org.apache.doris.datasource.hive.source.HiveScanNode; +import org.apache.doris.datasource.hudi.HudiPartitionUtils; import org.apache.doris.datasource.hudi.HudiSchemaCacheValue; import org.apache.doris.datasource.hudi.HudiUtils; import org.apache.doris.datasource.mvcc.MvccUtil; @@ -223,8 +224,8 @@ protected void doInitialize() throws UserException { fsView = Env.getCurrentEnv() .getExtMetaCacheMgr() - .getFsViewProcessor(hmsTable.getCatalog()) - .getFsView(hmsTable.getDbName(), hmsTable.getName(), hudiClient); + .hudi(hmsTable.getCatalog().getId()) + .getFsView(hmsTable.getOrBuildNameMapping()); // Todo: Get the current schema id of the table, instead of using -1. // In Be Parquet/Rrc reader, if `current table schema id == current file schema id`, then its // `table_info_node_ptr` will be `TableSchemaChangeHelper::ConstNode`. When using `ConstNode`, @@ -378,7 +379,7 @@ private List getIncrementalSplits() { List partitionNames = partitionColumns.isPresent() ? Arrays.asList(partitionColumns.get()) : Collections.emptyList(); return incrementalRelation.collectFileSlices().stream().map(fileSlice -> generateHudiSplit(fileSlice, - HudiPartitionProcessor.parsePartitionValues(partitionNames, fileSlice.getPartitionPath()), + HudiPartitionUtils.parsePartitionValues(partitionNames, fileSlice.getPartitionPath()), incrementalRelation.getEndTs())).collect(Collectors.toList()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java index 59f5525332e525..ded5ff679dc18c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java @@ -39,7 +39,6 @@ import java.util.List; import java.util.Map; -import java.util.Objects; public abstract class IcebergExternalCatalog extends ExternalCatalog { @@ -106,16 +105,10 @@ public void checkProperties() throws DdlException { @Override public void notifyPropertiesUpdated(Map updatedProps) { super.notifyPropertiesUpdated(updatedProps); - String tableCacheEnable = updatedProps.getOrDefault(ICEBERG_TABLE_CACHE_ENABLE, null); - String tableCacheTtl = updatedProps.getOrDefault(ICEBERG_TABLE_CACHE_TTL_SECOND, null); - String tableCacheCapacity = updatedProps.getOrDefault(ICEBERG_TABLE_CACHE_CAPACITY, null); - String manifestCacheEnable = updatedProps.getOrDefault(ICEBERG_MANIFEST_CACHE_ENABLE, null); - String manifestCacheCapacity = updatedProps.getOrDefault(ICEBERG_MANIFEST_CACHE_CAPACITY, null); - String manifestCacheTtl = updatedProps.getOrDefault(ICEBERG_MANIFEST_CACHE_TTL_SECOND, null); - if (Objects.nonNull(tableCacheEnable) || Objects.nonNull(tableCacheTtl) || Objects.nonNull(tableCacheCapacity) - || Objects.nonNull(manifestCacheEnable) || Objects.nonNull(manifestCacheCapacity) - || Objects.nonNull(manifestCacheTtl)) { - Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache(this).init(); + if (updatedProps.keySet().stream() + .anyMatch(key -> CacheSpec.isMetaCacheKeyForEngine(key, IcebergExternalMetaCache.ENGINE))) { + Env.getCurrentEnv().getExtMetaCacheMgr() + .removeCatalogByEngine(getId(), IcebergExternalMetaCache.ENGINE); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalMetaCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalMetaCache.java new file mode 100644 index 00000000000000..05378977443f49 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalMetaCache.java @@ -0,0 +1,289 @@ +// 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.datasource.iceberg; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.datasource.CacheException; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.doris.datasource.hive.HMSExternalCatalog; +import org.apache.doris.datasource.iceberg.cache.ManifestCacheValue; +import org.apache.doris.datasource.metacache.AbstractExternalMetaCache; +import org.apache.doris.datasource.metacache.CacheSpec; +import org.apache.doris.datasource.metacache.MetaCacheEntry; +import org.apache.doris.datasource.metacache.MetaCacheEntryDef; +import org.apache.doris.datasource.metacache.MetaCacheEntryInvalidation; +import org.apache.doris.mtmv.MTMVRelatedTableIf; + +import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestReader; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.view.View; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; + +/** + * Iceberg engine implementation of {@link AbstractExternalMetaCache}. + * + *

Registered entries: + *

    + *
  • {@code table}: loaded Iceberg {@link Table} instances per Doris table mapping, each + * memoizing its latest snapshot runtime projection
  • + *
  • {@code view}: loaded Iceberg {@link View} instances
  • + *
  • {@code manifest}: parsed manifest payload ({@link ManifestCacheValue}) keyed by + * manifest path and content type
  • + *
  • {@code schema}: schema cache keyed by table identity + schema id
  • + *
+ * + *

Manifest entry keys are path-based and intentionally not table-scoped. This allows + * shared manifests to reuse one cache entry across tables in the same catalog. + * + *

Invalidation behavior: + *

    + *
  • catalog invalidation clears all entries and drops Iceberg {@link ManifestFiles} IO cache
  • + *
  • db/table invalidation clears table/view/schema entries, while keeping manifest entries
  • + *
  • partition-level invalidation falls back to table-level invalidation
  • + *
+ */ +public class IcebergExternalMetaCache extends AbstractExternalMetaCache { + private static final Logger LOG = LogManager.getLogger(IcebergExternalMetaCache.class); + + public static final String ENGINE = "iceberg"; + public static final String ENTRY_TABLE = "table"; + public static final String ENTRY_VIEW = "view"; + public static final String ENTRY_MANIFEST = "manifest"; + public static final String ENTRY_SCHEMA = "schema"; + private static final long DEFAULT_MANIFEST_CACHE_CAPACITY = 100_000L; + + private final EntryHandle tableEntry; + private final EntryHandle viewEntry; + private final EntryHandle manifestEntry; + private final EntryHandle schemaEntry; + + public IcebergExternalMetaCache(ExecutorService refreshExecutor) { + super(ENGINE, refreshExecutor); + tableEntry = registerEntry(MetaCacheEntryDef.of(ENTRY_TABLE, NameMapping.class, IcebergTableCacheValue.class, + this::loadTableCacheValue, defaultEntryCacheSpec(), + MetaCacheEntryInvalidation.forNameMapping(nameMapping -> nameMapping))); + viewEntry = registerEntry(MetaCacheEntryDef.of(ENTRY_VIEW, NameMapping.class, View.class, this::loadView, + defaultEntryCacheSpec(), MetaCacheEntryInvalidation.forNameMapping(nameMapping -> nameMapping))); + manifestEntry = registerEntry(MetaCacheEntryDef.contextualOnly(ENTRY_MANIFEST, IcebergManifestEntryKey.class, + ManifestCacheValue.class, + CacheSpec.of(false, CacheSpec.CACHE_NO_TTL, DEFAULT_MANIFEST_CACHE_CAPACITY))); + schemaEntry = registerEntry(MetaCacheEntryDef.of(ENTRY_SCHEMA, IcebergSchemaCacheKey.class, + SchemaCacheValue.class, this::loadSchemaCacheValue, defaultSchemaCacheSpec(), + MetaCacheEntryInvalidation.forNameMapping(IcebergSchemaCacheKey::getNameMapping))); + } + + public Table getIcebergTable(ExternalTable dorisTable) { + NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); + return tableEntry.get(nameMapping.getCtlId()).get(nameMapping).getIcebergTable(); + } + + public IcebergSnapshotCacheValue getSnapshotCache(ExternalTable dorisTable) { + NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); + return tableEntry.get(nameMapping.getCtlId()).get(nameMapping).getLatestSnapshotCacheValue(); + } + + public List getSnapshotList(ExternalTable dorisTable) { + Table icebergTable = getIcebergTable(dorisTable); + List snapshots = com.google.common.collect.Lists.newArrayList(); + com.google.common.collect.Iterables.addAll(snapshots, icebergTable.snapshots()); + return snapshots; + } + + public View getIcebergView(ExternalTable dorisTable) { + NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); + return viewEntry.get(nameMapping.getCtlId()).get(nameMapping); + } + + public IcebergSchemaCacheValue getIcebergSchemaCacheValue(NameMapping nameMapping, long schemaId) { + IcebergSchemaCacheKey key = new IcebergSchemaCacheKey(nameMapping, schemaId); + SchemaCacheValue schemaCacheValue = schemaEntry.get(nameMapping.getCtlId()).get(key); + return (IcebergSchemaCacheValue) schemaCacheValue; + } + + public ManifestCacheValue getManifestCacheValue(ExternalTable dorisTable, + org.apache.iceberg.ManifestFile manifest, + Table icebergTable, + Consumer cacheHitRecorder) { + NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); + MetaCacheEntry manifestEntry = + this.manifestEntry.get(nameMapping.getCtlId()); + IcebergManifestEntryKey key = IcebergManifestEntryKey.of(manifest); + boolean hit = manifestEntry.getIfPresent(key) != null; + if (cacheHitRecorder != null) { + cacheHitRecorder.accept(hit); + } + return manifestEntry.get(key, ignored -> loadManifestCacheValue(manifest, icebergTable, key.getContent())); + } + + @Override + public void invalidateCatalog(long catalogId) { + dropManifestFileIoCacheForCatalog(catalogId); + super.invalidateCatalog(catalogId); + } + + @Override + public void invalidateCatalogEntries(long catalogId) { + dropManifestFileIoCacheForCatalog(catalogId); + super.invalidateCatalogEntries(catalogId); + } + + private IcebergTableCacheValue loadTableCacheValue(NameMapping nameMapping) { + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(nameMapping.getCtlId()); + if (catalog == null) { + throw new RuntimeException(String.format("Cannot find catalog %d when loading table %s/%s.", + nameMapping.getCtlId(), nameMapping.getLocalDbName(), nameMapping.getLocalTblName())); + } + + IcebergMetadataOps ops = resolveMetadataOps(catalog); + try { + Table table = ((ExternalCatalog) catalog).getExecutionAuthenticator() + .execute(() -> ops.loadTable(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName())); + ExternalTable dorisTable = findExternalTable(nameMapping, ENGINE); + return new IcebergTableCacheValue(table, () -> loadSnapshotProjection(dorisTable, table)); + } catch (Exception e) { + throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e), e); + } + } + + private View loadView(NameMapping nameMapping) { + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(nameMapping.getCtlId()); + if (!(catalog instanceof IcebergExternalCatalog)) { + return null; + } + IcebergMetadataOps ops = (IcebergMetadataOps) (((IcebergExternalCatalog) catalog).getMetadataOps()); + try { + return ((ExternalCatalog) catalog).getExecutionAuthenticator().execute( + () -> ops.loadView(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName())); + } catch (Exception e) { + throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e), e); + } + } + + private ManifestCacheValue loadManifestCacheValue(org.apache.iceberg.ManifestFile manifest, Table icebergTable, + ManifestContent content) { + if (manifest == null || icebergTable == null) { + String manifestPath = manifest == null ? "null" : manifest.path(); + throw new CacheException("Manifest cache loader context is missing for %s", + null, manifestPath); + } + try { + if (content == ManifestContent.DELETES) { + return ManifestCacheValue.forDeleteFiles( + loadDeleteFiles(manifest, icebergTable)); + } + return ManifestCacheValue.forDataFiles(loadDataFiles(manifest, icebergTable)); + } catch (IOException e) { + throw new CacheException("Failed to read manifest %s", e, manifest.path()); + } + } + + private SchemaCacheValue loadSchemaCacheValue(IcebergSchemaCacheKey key) { + ExternalTable dorisTable = findExternalTable(key.getNameMapping(), ENGINE); + return dorisTable.initSchemaAndUpdateTime(key).orElseThrow(() -> + new CacheException("failed to load iceberg schema cache value for: %s.%s.%s, schemaId: %s", + null, key.getNameMapping().getCtlId(), key.getNameMapping().getLocalDbName(), + key.getNameMapping().getLocalTblName(), key.getSchemaId())); + } + + private IcebergSnapshotCacheValue loadSnapshotProjection(ExternalTable dorisTable, Table icebergTable) { + if (!(dorisTable instanceof MTMVRelatedTableIf)) { + throw new RuntimeException(String.format("Table %s.%s is not a valid MTMV related table.", + dorisTable.getDbName(), dorisTable.getName())); + } + try { + MTMVRelatedTableIf table = (MTMVRelatedTableIf) dorisTable; + IcebergSnapshot latestIcebergSnapshot = IcebergUtils.getLatestIcebergSnapshot(icebergTable); + IcebergPartitionInfo icebergPartitionInfo; + if (!table.isValidRelatedTable()) { + icebergPartitionInfo = IcebergPartitionInfo.empty(); + } else { + icebergPartitionInfo = IcebergUtils.loadPartitionInfo(dorisTable, icebergTable, + latestIcebergSnapshot.getSnapshotId(), latestIcebergSnapshot.getSchemaId()); + } + return new IcebergSnapshotCacheValue(icebergPartitionInfo, latestIcebergSnapshot); + } catch (AnalysisException e) { + throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e), e); + } + } + + private IcebergMetadataOps resolveMetadataOps(CatalogIf catalog) { + if (catalog instanceof HMSExternalCatalog) { + return ((HMSExternalCatalog) catalog).getIcebergMetadataOps(); + } else if (catalog instanceof IcebergExternalCatalog) { + return (IcebergMetadataOps) (((IcebergExternalCatalog) catalog).getMetadataOps()); + } + throw new RuntimeException("Only support 'hms' and 'iceberg' type for iceberg table"); + } + + @Override + protected Map catalogPropertyCompatibilityMap() { + return singleCompatibilityMap(ExternalCatalog.SCHEMA_CACHE_TTL_SECOND, ENTRY_SCHEMA); + } + + private List loadDataFiles(org.apache.iceberg.ManifestFile manifest, Table table) + throws IOException { + List dataFiles = com.google.common.collect.Lists.newArrayList(); + try (ManifestReader reader = ManifestFiles.read(manifest, table.io())) { + for (org.apache.iceberg.DataFile dataFile : reader) { + dataFiles.add(dataFile.copy()); + } + } + return dataFiles; + } + + private List loadDeleteFiles(org.apache.iceberg.ManifestFile manifest, Table table) + throws IOException { + List deleteFiles = com.google.common.collect.Lists.newArrayList(); + try (ManifestReader reader = ManifestFiles.readDeleteManifest(manifest, + table.io(), table.specs())) { + for (org.apache.iceberg.DeleteFile deleteFile : reader) { + deleteFiles.add(deleteFile.copy()); + } + } + return deleteFiles; + } + + private void dropManifestFileIoCacheForCatalog(long catalogId) { + tableEntry.get(catalogId).forEach((key, value) -> dropManifestFileIoCache(value)); + } + + private void dropManifestFileIoCache(IcebergTableCacheValue tableCacheValue) { + try { + ManifestFiles.dropCache(tableCacheValue.getIcebergTable().io()); + } catch (Exception e) { + LOG.warn("Failed to drop iceberg manifest files cache", e); + } + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java index bae9462b31ebd1..aceb2a2408a74a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java @@ -25,12 +25,13 @@ import org.apache.doris.catalog.PartitionType; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; -import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey; import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.SchemaCacheKey; import org.apache.doris.datasource.SchemaCacheValue; import org.apache.doris.datasource.mvcc.EmptyMvccSnapshot; import org.apache.doris.datasource.mvcc.MvccSnapshot; import org.apache.doris.datasource.mvcc.MvccTable; +import org.apache.doris.datasource.mvcc.MvccUtil; import org.apache.doris.datasource.systable.IcebergSysTable; import org.apache.doris.datasource.systable.SysTable; import org.apache.doris.mtmv.MTMVBaseTableIf; @@ -77,6 +78,11 @@ public IcebergExternalTable(long id, String name, String remoteName, IcebergExte super(id, name, remoteName, catalog, db, TableType.ICEBERG_EXTERNAL_TABLE); } + @Override + public String getMetaCacheEngine() { + return IcebergExternalMetaCache.ENGINE; + } + public String getIcebergCatalogType() { return ((IcebergExternalCatalog) catalog).getIcebergCatalogType(); } @@ -95,6 +101,13 @@ public Optional initSchema(SchemaCacheKey key) { return IcebergUtils.loadSchemaCacheValue(this, ((IcebergSchemaCacheKey) key).getSchemaId(), isView); } + @Override + public Optional getSchemaCacheValue() { + IcebergSnapshotCacheValue snapshotValue = IcebergUtils.getSnapshotCacheValue( + MvccUtil.getSnapshotFromContext(this), this); + return Optional.of(IcebergUtils.getSchemaCacheValue(this, snapshotValue)); + } + @Override public TTableDescriptor toThrift() { List schema = getFullSchema(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergManifestEntryKey.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergManifestEntryKey.java new file mode 100644 index 00000000000000..3d6499333be136 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergManifestEntryKey.java @@ -0,0 +1,69 @@ +// 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.datasource.iceberg; + +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; + +import java.util.Objects; + +/** + * Cache key for one iceberg manifest entry. + * + *

This key only contains stable identity dimensions (manifest path + content type). + * Runtime loader context (manifest instance, table instance) must not be stored here. + */ +public class IcebergManifestEntryKey { + private final String manifestPath; + private final ManifestContent content; + + public IcebergManifestEntryKey(String manifestPath, ManifestContent content) { + this.manifestPath = Objects.requireNonNull(manifestPath, "manifestPath can not be null"); + this.content = Objects.requireNonNull(content, "content can not be null"); + } + + public static IcebergManifestEntryKey of(ManifestFile manifest) { + return new IcebergManifestEntryKey(manifest.path(), manifest.content()); + } + + public String getManifestPath() { + return manifestPath; + } + + public ManifestContent getContent() { + return content; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof IcebergManifestEntryKey)) { + return false; + } + IcebergManifestEntryKey that = (IcebergManifestEntryKey) o; + return Objects.equals(manifestPath, that.manifestPath) + && content == that.content; + } + + @Override + public int hashCode() { + return Objects.hash(manifestPath, content); + } +} 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 deleted file mode 100644 index 40c8ba29184183..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java +++ /dev/null @@ -1,309 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.datasource.iceberg; - -import org.apache.doris.catalog.Env; -import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.CacheFactory; -import org.apache.doris.common.Config; -import org.apache.doris.datasource.CatalogIf; -import org.apache.doris.datasource.ExternalCatalog; -import org.apache.doris.datasource.ExternalMetaCacheMgr; -import org.apache.doris.datasource.ExternalTable; -import org.apache.doris.datasource.NameMapping; -import org.apache.doris.datasource.hive.HMSExternalCatalog; -import org.apache.doris.datasource.iceberg.cache.IcebergManifestCache; -import org.apache.doris.datasource.metacache.CacheSpec; -import org.apache.doris.mtmv.MTMVRelatedTableIf; - -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.commons.lang3.exception.ExceptionUtils; -import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.view.View; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.jetbrains.annotations.NotNull; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.OptionalLong; -import java.util.concurrent.ExecutorService; - -public class IcebergMetadataCache { - private static final Logger LOG = LogManager.getLogger(IcebergMetadataCache.class); - private final ExecutorService executor; - private final ExternalCatalog catalog; - private LoadingCache tableCache; - private LoadingCache viewCache; - private IcebergManifestCache manifestCache; - - public IcebergMetadataCache(ExternalCatalog catalog, ExecutorService executor) { - this.executor = executor; - this.catalog = catalog; - init(); - } - - public void init() { - CacheSpec tableCacheSpec = resolveTableCacheSpec(); - CacheFactory tableCacheFactory = new CacheFactory( - CacheSpec.toExpireAfterAccess(tableCacheSpec.getTtlSecond()), - OptionalLong.empty(), - tableCacheSpec.getCapacity(), - true, - null); - this.tableCache = tableCacheFactory.buildCache(this::loadTableCacheValue, executor); - this.viewCache = tableCacheFactory.buildCache(this::loadView, executor); - - CacheSpec manifestCacheSpec = resolveManifestCacheSpec(); - this.manifestCache = new IcebergManifestCache(manifestCacheSpec.getCapacity(), - manifestCacheSpec.getTtlSecond()); - } - - private CacheSpec resolveTableCacheSpec() { - return CacheSpec.fromProperties(catalog.getProperties(), - IcebergExternalCatalog.ICEBERG_TABLE_CACHE_ENABLE, true, - IcebergExternalCatalog.ICEBERG_TABLE_CACHE_TTL_SECOND, - Config.external_cache_expire_time_seconds_after_access, - IcebergExternalCatalog.ICEBERG_TABLE_CACHE_CAPACITY, - Config.max_external_table_cache_num); - } - - private CacheSpec resolveManifestCacheSpec() { - return CacheSpec.fromProperties(catalog.getProperties(), - IcebergExternalCatalog.ICEBERG_MANIFEST_CACHE_ENABLE, - IcebergExternalCatalog.DEFAULT_ICEBERG_MANIFEST_CACHE_ENABLE, - IcebergExternalCatalog.ICEBERG_MANIFEST_CACHE_TTL_SECOND, - IcebergExternalCatalog.DEFAULT_ICEBERG_MANIFEST_CACHE_TTL_SECOND, - IcebergExternalCatalog.ICEBERG_MANIFEST_CACHE_CAPACITY, - IcebergExternalCatalog.DEFAULT_ICEBERG_MANIFEST_CACHE_CAPACITY); - } - - public Table getIcebergTable(ExternalTable dorisTable) { - IcebergMetadataCacheKey key = new IcebergMetadataCacheKey(dorisTable.getOrBuildNameMapping()); - return tableCache.get(key).getIcebergTable(); - } - - public Table getIcebergTable(IcebergMetadataCacheKey key) { - return tableCache.get(key).getIcebergTable(); - } - - public IcebergSnapshotCacheValue getSnapshotCache(ExternalTable dorisTable) { - IcebergMetadataCacheKey key = new IcebergMetadataCacheKey(dorisTable.getOrBuildNameMapping()); - IcebergTableCacheValue tableCacheValue = tableCache.get(key); - return tableCacheValue.getSnapshotCacheValue(() -> loadSnapshot(dorisTable, tableCacheValue.getIcebergTable())); - } - - public List getSnapshotList(ExternalTable dorisTable) { - Table icebergTable = getIcebergTable(dorisTable); - List snaps = Lists.newArrayList(); - Iterables.addAll(snaps, icebergTable.snapshots()); - return snaps; - } - - public IcebergManifestCache getManifestCache() { - return manifestCache; - } - - @NotNull - private IcebergTableCacheValue loadTableCacheValue(IcebergMetadataCacheKey key) { - NameMapping nameMapping = key.nameMapping; - CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(nameMapping.getCtlId()); - if (catalog == null) { - throw new RuntimeException(String.format("Cannot find catalog %d when loading table %s/%s.", - nameMapping.getCtlId(), nameMapping.getLocalDbName(), nameMapping.getLocalTblName())); - } - IcebergMetadataOps ops; - if (catalog instanceof HMSExternalCatalog) { - ops = ((HMSExternalCatalog) catalog).getIcebergMetadataOps(); - } else if (catalog instanceof IcebergExternalCatalog) { - ops = (IcebergMetadataOps) (((IcebergExternalCatalog) catalog).getMetadataOps()); - } else { - throw new RuntimeException("Only support 'hms' and 'iceberg' type for iceberg table"); - } - try { - if (LOG.isDebugEnabled()) { - LOG.debug("load iceberg table {}", nameMapping, new Exception()); - } - Table table = ((ExternalCatalog) catalog).getExecutionAuthenticator() - .execute(() - -> ops.loadTable(nameMapping.getRemoteDbName(), nameMapping.getRemoteTblName())); - return new IcebergTableCacheValue(table); - } catch (Exception e) { - throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e), e); - } - - } - - @NotNull - private IcebergSnapshotCacheValue loadSnapshot(ExternalTable dorisTable, Table icebergTable) { - if (!(dorisTable instanceof MTMVRelatedTableIf)) { - throw new RuntimeException(String.format("Table %s.%s is not a valid MTMV related table.", - dorisTable.getDbName(), dorisTable.getName())); - } - - try { - MTMVRelatedTableIf table = (MTMVRelatedTableIf) dorisTable; - IcebergSnapshot latestIcebergSnapshot = IcebergUtils.getLatestIcebergSnapshot(icebergTable); - IcebergPartitionInfo icebergPartitionInfo; - if (!table.isValidRelatedTable()) { - icebergPartitionInfo = IcebergPartitionInfo.empty(); - } else { - icebergPartitionInfo = IcebergUtils.loadPartitionInfo(dorisTable, icebergTable, - latestIcebergSnapshot.getSnapshotId(), latestIcebergSnapshot.getSchemaId()); - } - return new IcebergSnapshotCacheValue(icebergPartitionInfo, latestIcebergSnapshot); - } catch (AnalysisException e) { - throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e), e); - } - } - - public void invalidateCatalogCache(long catalogId) { - if (LOG.isDebugEnabled()) { - LOG.debug("invalidate all iceberg table cache when invalidating catalog {}", catalogId); - } - // Invalidate all entries related to the catalog - tableCache.invalidateAll(); - viewCache.invalidateAll(); - manifestCache.invalidateAll(); - } - - public void invalidateTableCache(ExternalTable dorisTable) { - IcebergMetadataCacheKey key = IcebergMetadataCacheKey.of(dorisTable.getOrBuildNameMapping()); - IcebergTableCacheValue tableCacheValue = tableCache.getIfPresent(key); - if (tableCacheValue != null) { - invalidateTableCache(key, tableCacheValue); - } else { - invalidateTableCacheByLocalName(dorisTable); - } - } - - private void invalidateTableCache(IcebergMetadataCacheKey key, IcebergTableCacheValue tableCacheValue) { - ManifestFiles.dropCache(tableCacheValue.getIcebergTable().io()); - if (LOG.isDebugEnabled()) { - LOG.debug("invalidate iceberg table cache {}", key.nameMapping, new Exception()); - } - tableCache.invalidate(key); - viewCache.invalidate(key); - } - - private void invalidateTableCacheByLocalName(ExternalTable dorisTable) { - String dbName = dorisTable.getDbName(); - String tblName = dorisTable.getName(); - tableCache.asMap().entrySet().stream() - .filter(entry -> entry.getKey().nameMapping.getLocalDbName().equals(dbName) - && entry.getKey().nameMapping.getLocalTblName().equals(tblName)) - .forEach(entry -> invalidateTableCache(entry.getKey(), entry.getValue())); - viewCache.asMap().keySet().stream() - .filter(key -> key.nameMapping.getLocalDbName().equals(dbName) - && key.nameMapping.getLocalTblName().equals(tblName)) - .forEach(viewCache::invalidate); - } - - public void invalidateDbCache(long catalogId, String dbName) { - tableCache.asMap().entrySet().stream() - .filter(entry -> entry.getKey().nameMapping.getLocalDbName().equals(dbName)) - .forEach(entry -> { - ManifestFiles.dropCache(entry.getValue().getIcebergTable().io()); - if (LOG.isDebugEnabled()) { - LOG.debug("invalidate iceberg table cache {} when invalidating db cache", - entry.getKey().nameMapping, new Exception()); - } - tableCache.invalidate(entry.getKey()); - }); - viewCache.asMap().keySet().stream() - .filter(key -> key.nameMapping.getLocalDbName().equals(dbName)) - .forEach(viewCache::invalidate); - } - - private static void initIcebergTableFileIO(Table table, Map props) { - Map ioConf = new HashMap<>(); - table.properties().forEach((key, value) -> { - if (key.startsWith("io.")) { - ioConf.put(key, value); - } - }); - - // This `initialize` method will directly override the properties as a whole, - // so we need to merge the table's io-related properties with the doris's catalog-related properties - props.putAll(ioConf); - table.io().initialize(props); - } - - static class IcebergMetadataCacheKey { - NameMapping nameMapping; - - private IcebergMetadataCacheKey(NameMapping nameMapping) { - this.nameMapping = nameMapping; - } - - private static IcebergMetadataCacheKey of(NameMapping nameMapping) { - return new IcebergMetadataCacheKey(nameMapping); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - IcebergMetadataCacheKey that = (IcebergMetadataCacheKey) o; - return nameMapping.equals(that.nameMapping); - } - - @Override - public int hashCode() { - return nameMapping.hashCode(); - } - } - - public Map> getCacheStats() { - Map> res = Maps.newHashMap(); - res.put("iceberg_table_cache", ExternalMetaCacheMgr.getCacheStats(tableCache.stats(), - tableCache.estimatedSize())); - return res; - } - - private View loadView(IcebergMetadataCacheKey key) { - IcebergMetadataOps ops; - CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(key.nameMapping.getCtlId()); - if (catalog instanceof IcebergExternalCatalog) { - ops = (IcebergMetadataOps) (((IcebergExternalCatalog) catalog).getMetadataOps()); - } else { - return null; - } - try { - return ((ExternalCatalog) catalog).getExecutionAuthenticator().execute(() -> - ops.loadView(key.nameMapping.getRemoteDbName(), key.nameMapping.getRemoteTblName())); - } catch (Exception e) { - throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e), e); - } - } - - public View getIcebergView(ExternalTable dorisTable) { - IcebergMetadataCacheKey key = new IcebergMetadataCacheKey(dorisTable.getOrBuildNameMapping()); - return viewCache.get(key); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSchemaCacheKey.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSchemaCacheKey.java index 0f5d15e4f6fb48..7c2d09511a2c93 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSchemaCacheKey.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSchemaCacheKey.java @@ -17,8 +17,8 @@ package org.apache.doris.datasource.iceberg; -import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey; import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.SchemaCacheKey; import com.google.common.base.Objects; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTableCacheValue.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTableCacheValue.java index b133a9125223ec..cdef77346ade27 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTableCacheValue.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTableCacheValue.java @@ -17,33 +17,25 @@ package org.apache.doris.datasource.iceberg; +import com.google.common.base.Suppliers; import org.apache.iceberg.Table; import java.util.function.Supplier; public class IcebergTableCacheValue { private final Table icebergTable; + private final Supplier latestSnapshotCacheValue; - private volatile boolean snapshotCacheLoaded; - private volatile IcebergSnapshotCacheValue snapshotCacheValue; - - public IcebergTableCacheValue(Table icebergTable) { + public IcebergTableCacheValue(Table icebergTable, Supplier latestSnapshotCacheValue) { this.icebergTable = icebergTable; + this.latestSnapshotCacheValue = Suppliers.memoize(latestSnapshotCacheValue::get); } public Table getIcebergTable() { return icebergTable; } - public IcebergSnapshotCacheValue getSnapshotCacheValue(Supplier loader) { - if (!snapshotCacheLoaded) { - synchronized (this) { - if (!snapshotCacheLoaded) { - snapshotCacheValue = loader.get(); - snapshotCacheLoaded = true; - } - } - } - return snapshotCacheValue; + public IcebergSnapshotCacheValue getLatestSnapshotCacheValue() { + return latestSnapshotCacheValue.get(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java index 75db779d157549..48a0c96a3a007a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java @@ -51,12 +51,9 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; import org.apache.doris.common.util.TimeUtils; -import org.apache.doris.datasource.CacheException; import org.apache.doris.datasource.ExternalCatalog; -import org.apache.doris.datasource.ExternalSchemaCache; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.SchemaCacheValue; -import org.apache.doris.datasource.iceberg.cache.IcebergManifestCache; import org.apache.doris.datasource.iceberg.source.IcebergTableQueryInfo; import org.apache.doris.datasource.metacache.CacheSpec; import org.apache.doris.datasource.mvcc.MvccSnapshot; @@ -733,16 +730,16 @@ private static String serializePartitionValue(org.apache.iceberg.types.Type type } public static Table getIcebergTable(ExternalTable dorisTable) { - return icebergMetadataCache(dorisTable.getCatalog()).getIcebergTable(dorisTable); + return icebergExternalMetaCache(dorisTable).getIcebergTable(dorisTable); } - // Centralize cache access to keep call sites consistent and easy to understand. - private static IcebergMetadataCache icebergMetadataCache(ExternalCatalog catalog) { - return Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache(catalog); + private static IcebergExternalMetaCache icebergExternalMetaCache(ExternalCatalog catalog) { + Preconditions.checkNotNull(catalog, "catalog can not be null"); + return Env.getCurrentEnv().getExtMetaCacheMgr().iceberg(catalog.getId()); } - private static ExternalSchemaCache schemaCache(ExternalCatalog catalog) { - return Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); + private static IcebergExternalMetaCache icebergExternalMetaCache(ExternalTable table) { + return icebergExternalMetaCache(table.getCatalog()); } public static org.apache.iceberg.types.Type dorisTypeToIcebergType(Type type) { @@ -1219,15 +1216,10 @@ public static boolean isIcebergBranchOrTag(Optional scanParams) return false; } - // read schema from external schema cache + // read schema from iceberg.schema entry public static IcebergSchemaCacheValue getSchemaCacheValue(ExternalTable dorisTable, long schemaId) { - Optional schemaCacheValue = schemaCache(dorisTable.getCatalog()).getSchemaValue( - new IcebergSchemaCacheKey(dorisTable.getOrBuildNameMapping(), schemaId)); - if (!schemaCacheValue.isPresent()) { - throw new CacheException("failed to getSchema for: %s.%s.%s.%s", - null, dorisTable.getCatalog().getName(), dorisTable.getDbName(), dorisTable.getName(), schemaId); - } - return (IcebergSchemaCacheValue) schemaCacheValue.get(); + return icebergExternalMetaCache(dorisTable) + .getIcebergSchemaCacheValue(dorisTable.getOrBuildNameMapping(), schemaId); } public static IcebergSnapshot getLatestIcebergSnapshot(Table table) { @@ -1497,7 +1489,7 @@ public static IcebergSchemaCacheValue getSchemaCacheValue(ExternalTable dorisTab } public static IcebergSnapshotCacheValue getLatestSnapshotCacheValue(ExternalTable dorisTable) { - return icebergMetadataCache(dorisTable.getCatalog()).getSnapshotCache(dorisTable); + return icebergExternalMetaCache(dorisTable).getSnapshotCache(dorisTable); } public static IcebergSnapshotCacheValue getSnapshotCacheValue(Optional snapshot, @@ -1545,7 +1537,7 @@ public static Map getIcebergPartitionItems(Optional loadSchemaCacheValue( @@ -1584,18 +1576,15 @@ public static String showCreateView(IcebergExternalTable icebergExternalTable) { icebergExternalTable.getViewText(); } - public static IcebergManifestCache getManifestCache(ExternalCatalog catalog) { - return icebergMetadataCache(catalog).getManifestCache(); - } - public static boolean isManifestCacheEnabled(ExternalCatalog catalog) { - CacheSpec spec = CacheSpec.fromProperties(catalog.getProperties(), - IcebergExternalCatalog.ICEBERG_MANIFEST_CACHE_ENABLE, - IcebergExternalCatalog.DEFAULT_ICEBERG_MANIFEST_CACHE_ENABLE, - IcebergExternalCatalog.ICEBERG_MANIFEST_CACHE_TTL_SECOND, - IcebergExternalCatalog.DEFAULT_ICEBERG_MANIFEST_CACHE_TTL_SECOND, - IcebergExternalCatalog.ICEBERG_MANIFEST_CACHE_CAPACITY, - IcebergExternalCatalog.DEFAULT_ICEBERG_MANIFEST_CACHE_CAPACITY); + CacheSpec spec = CacheSpec.fromProperties(catalog.getProperties(), CacheSpec.propertySpecBuilder() + .enable(IcebergExternalCatalog.ICEBERG_MANIFEST_CACHE_ENABLE, + IcebergExternalCatalog.DEFAULT_ICEBERG_MANIFEST_CACHE_ENABLE) + .ttl(IcebergExternalCatalog.ICEBERG_MANIFEST_CACHE_TTL_SECOND, + IcebergExternalCatalog.DEFAULT_ICEBERG_MANIFEST_CACHE_TTL_SECOND) + .capacity(IcebergExternalCatalog.ICEBERG_MANIFEST_CACHE_CAPACITY, + IcebergExternalCatalog.DEFAULT_ICEBERG_MANIFEST_CACHE_CAPACITY) + .build()); return CacheSpec.isCacheEnabled(spec.isEnable(), spec.getTtlSecond(), spec.getCapacity()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/cache/IcebergManifestCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/cache/IcebergManifestCache.java deleted file mode 100644 index 6016b2ab7e997e..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/cache/IcebergManifestCache.java +++ /dev/null @@ -1,84 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.datasource.iceberg.cache; - -import org.apache.doris.common.CacheFactory; -import org.apache.doris.datasource.CacheException; -import org.apache.doris.datasource.metacache.CacheSpec; - -import com.github.benmanes.caffeine.cache.CacheLoader; -import com.github.benmanes.caffeine.cache.LoadingCache; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.Optional; -import java.util.concurrent.Callable; - -/** - * A lightweight manifest cache that stores parsed DataFile/DeleteFile lists per manifest. - */ -public class IcebergManifestCache { - private static final Logger LOG = LogManager.getLogger(IcebergManifestCache.class); - - private final LoadingCache cache; - - public IcebergManifestCache(long capacity, long ttlSec) { - CacheFactory cacheFactory = new CacheFactory( - CacheSpec.toExpireAfterAccess(ttlSec), - java.util.OptionalLong.empty(), - capacity, - true, - null); - cache = cacheFactory.buildCache(new CacheLoader() { - @Override - public ManifestCacheValue load(ManifestCacheKey key) { - throw new CacheException("Manifest cache loader should be provided explicitly for key %s", null, key); - } - }); - } - - public ManifestCacheValue get(ManifestCacheKey key, Callable loader) { - try { - return cache.get(key, ignored -> { - try { - return loader.call(); - } catch (Exception e) { - throw new RuntimeException(e); - } - }); - } catch (Exception e) { - throw new CacheException("Failed to load manifest cache for key %s", e, key); - } - } - - public Optional peek(ManifestCacheKey key) { - return Optional.ofNullable(cache.getIfPresent(key)); - } - - public void invalidateByPath(String path) { - cache.invalidate(buildKey(path)); - } - - public void invalidateAll() { - cache.invalidateAll(); - } - - public ManifestCacheKey buildKey(String path) { - return new ManifestCacheKey(path); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/cache/IcebergManifestCacheLoader.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/cache/IcebergManifestCacheLoader.java index 8ec14a50e6af18..bd76e7e6424908 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/cache/IcebergManifestCacheLoader.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/cache/IcebergManifestCacheLoader.java @@ -17,102 +17,38 @@ package org.apache.doris.datasource.iceberg.cache; -import org.apache.doris.datasource.CacheException; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.iceberg.IcebergExternalMetaCache; -import com.google.common.collect.Lists; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.ManifestReader; import org.apache.iceberg.Table; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import java.io.IOException; -import java.util.List; -import java.util.Optional; import java.util.function.Consumer; /** * Helper to load manifest content and populate the manifest cache. */ public class IcebergManifestCacheLoader { - private static final Logger LOG = LogManager.getLogger(IcebergManifestCacheLoader.class); - private IcebergManifestCacheLoader() { } - public static ManifestCacheValue loadDataFilesWithCache(IcebergManifestCache cache, ManifestFile manifest, - Table table) { - return loadDataFilesWithCache(cache, manifest, table, null); - } - - public static ManifestCacheValue loadDataFilesWithCache(IcebergManifestCache cache, ManifestFile manifest, - Table table, Consumer cacheHitRecorder) { - return loadWithCache(cache, manifest, cacheHitRecorder, () -> loadDataFiles(manifest, table)); - } - - public static ManifestCacheValue loadDeleteFilesWithCache(IcebergManifestCache cache, ManifestFile manifest, - Table table) { - return loadDeleteFilesWithCache(cache, manifest, table, null); - } - - public static ManifestCacheValue loadDeleteFilesWithCache(IcebergManifestCache cache, ManifestFile manifest, - Table table, Consumer cacheHitRecorder) { - return loadWithCache(cache, manifest, cacheHitRecorder, () -> loadDeleteFiles(manifest, table)); - } - - private static ManifestCacheValue loadDataFiles(ManifestFile manifest, Table table) { - List dataFiles = Lists.newArrayList(); - try (ManifestReader reader = ManifestFiles.read(manifest, table.io())) { - // ManifestReader implements CloseableIterable, iterate directly - for (DataFile dataFile : reader) { - dataFiles.add(dataFile.copy()); - } - } catch (IOException e) { - LOG.warn("Failed to read data manifest {}", manifest.path(), e); - throw new CacheException("Failed to read data manifest %s", e, manifest.path()); - } - return ManifestCacheValue.forDataFiles(dataFiles); - } - - private static ManifestCacheValue loadDeleteFiles(ManifestFile manifest, Table table) { - List deleteFiles = Lists.newArrayList(); - try (ManifestReader reader = ManifestFiles.readDeleteManifest(manifest, table.io(), - table.specs())) { - // ManifestReader implements CloseableIterable, iterate directly - for (DeleteFile deleteFile : reader) { - deleteFiles.add(deleteFile.copy()); - } - } catch (IOException e) { - LOG.warn("Failed to read delete manifest {}", manifest.path(), e); - throw new CacheException("Failed to read delete manifest %s", e, manifest.path()); - } - return ManifestCacheValue.forDeleteFiles(deleteFiles); + public static ManifestCacheValue loadDataFilesWithCache(IcebergExternalMetaCache cache, ExternalTable dorisTable, + ManifestFile manifest, Table table) { + return loadDataFilesWithCache(cache, dorisTable, manifest, table, null); } - private static ManifestCacheValue loadWithCache(IcebergManifestCache cache, ManifestFile manifest, - Consumer cacheHitRecorder, Loader loader) { - ManifestCacheKey key = buildKey(cache, manifest); - Optional cached = cache.peek(key); - boolean cacheHit = cached.isPresent(); - if (cacheHitRecorder != null) { - cacheHitRecorder.accept(cacheHit); - } - if (cacheHit) { - return cached.get(); - } - return cache.get(key, loader::load); + public static ManifestCacheValue loadDataFilesWithCache(IcebergExternalMetaCache cache, ExternalTable dorisTable, + ManifestFile manifest, Table table, Consumer cacheHitRecorder) { + return cache.getManifestCacheValue(dorisTable, manifest, table, cacheHitRecorder); } - @FunctionalInterface - private interface Loader { - ManifestCacheValue load(); + public static ManifestCacheValue loadDeleteFilesWithCache(IcebergExternalMetaCache cache, + ExternalTable dorisTable, ManifestFile manifest, Table table) { + return loadDeleteFilesWithCache(cache, dorisTable, manifest, table, null); } - private static ManifestCacheKey buildKey(IcebergManifestCache cache, ManifestFile manifest) { - // Iceberg manifest files are immutable, so path uniquely identifies a manifest - return cache.buildKey(manifest.path()); + public static ManifestCacheValue loadDeleteFilesWithCache(IcebergExternalMetaCache cache, + ExternalTable dorisTable, ManifestFile manifest, Table table, Consumer cacheHitRecorder) { + return cache.getManifestCacheValue(dorisTable, manifest, table, cacheHitRecorder); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergApiSource.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergApiSource.java index 18d490eceb8e5b..c233f732106a29 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergApiSource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergApiSource.java @@ -18,7 +18,6 @@ package org.apache.doris.datasource.iceberg.source; import org.apache.doris.analysis.TupleDescriptor; -import org.apache.doris.catalog.Env; import org.apache.doris.catalog.TableIf; import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.datasource.ExternalCatalog; @@ -54,8 +53,7 @@ public IcebergApiSource(IcebergExternalTable table, TupleDescriptor desc, throw new UnsupportedOperationException("IcebergApiSource does not support view"); } this.icebergExtTable = table; - this.originTable = Env.getCurrentEnv().getExtMetaCacheMgr() - .getIcebergMetadataCache(table.getCatalog()).getIcebergTable(icebergExtTable); + this.originTable = IcebergUtils.getIcebergTable(icebergExtTable); this.desc = desc; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergHMSSource.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergHMSSource.java index 8057e970b389db..a87b5408d5938b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergHMSSource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergHMSSource.java @@ -18,7 +18,6 @@ package org.apache.doris.datasource.iceberg.source; import org.apache.doris.analysis.TupleDescriptor; -import org.apache.doris.catalog.Env; import org.apache.doris.catalog.TableIf; import org.apache.doris.common.DdlException; import org.apache.doris.common.MetaNotFoundException; @@ -35,9 +34,7 @@ public class IcebergHMSSource implements IcebergSource { public IcebergHMSSource(HMSExternalTable hmsTable, TupleDescriptor desc) { this.hmsTable = hmsTable; this.desc = desc; - this.icebergTable = - Env.getCurrentEnv().getExtMetaCacheMgr() - .getIcebergMetadataCache(hmsTable.getCatalog()).getIcebergTable(hmsTable); + this.icebergTable = IcebergUtils.getIcebergTable(hmsTable); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java index 053908bdd43a5b..8331c2b74f38ef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java @@ -37,9 +37,9 @@ import org.apache.doris.datasource.credentials.VendedCredentialsFactory; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; +import org.apache.doris.datasource.iceberg.IcebergExternalMetaCache; import org.apache.doris.datasource.iceberg.IcebergExternalTable; import org.apache.doris.datasource.iceberg.IcebergUtils; -import org.apache.doris.datasource.iceberg.cache.IcebergManifestCache; import org.apache.doris.datasource.iceberg.cache.IcebergManifestCacheLoader; import org.apache.doris.datasource.iceberg.cache.ManifestCacheValue; import org.apache.doris.datasource.iceberg.profile.IcebergMetricsReporter; @@ -516,7 +516,11 @@ private CloseableIterable planFileScanTaskWithManifestCache(TableS } // Initialize manifest cache for efficient manifest file access - IcebergManifestCache cache = IcebergUtils.getManifestCache(source.getCatalog()); + IcebergExternalMetaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().iceberg(source.getCatalog().getId()); + if (!(source.getTargetTable() instanceof ExternalTable)) { + throw new RuntimeException("Iceberg scan target table is not an external table"); + } + ExternalTable targetExternalTable = (ExternalTable) source.getTargetTable(); // Convert query conjuncts to Iceberg filter expression // This combines all predicates with AND logic for partition/file pruning @@ -560,8 +564,8 @@ private CloseableIterable planFileScanTaskWithManifestCache(TableS continue; } // Load delete files from cache (or from storage if not cached) - ManifestCacheValue value = IcebergManifestCacheLoader.loadDeleteFilesWithCache(cache, manifest, - icebergTable, this::recordManifestCacheAccess); + ManifestCacheValue value = IcebergManifestCacheLoader.loadDeleteFilesWithCache(cache, + targetExternalTable, manifest, icebergTable, this::recordManifestCacheAccess); deleteFiles.addAll(value.getDeleteFiles()); } @@ -593,8 +597,8 @@ private CloseableIterable planFileScanTaskWithManifestCache(TableS } // Load data files from cache (or from storage if not cached) - ManifestCacheValue value = IcebergManifestCacheLoader.loadDataFilesWithCache(cache, manifest, - icebergTable, this::recordManifestCacheAccess); + ManifestCacheValue value = IcebergManifestCacheLoader.loadDataFilesWithCache(cache, + targetExternalTable, manifest, icebergTable, this::recordManifestCacheAccess); // Process each data file in the manifest for (org.apache.iceberg.DataFile dataFile : value.getDataFiles()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/lakesoul/LakeSoulExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/lakesoul/LakeSoulExternalCatalog.java index 465f2c78382212..c374d75d0ea374 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/lakesoul/LakeSoulExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/lakesoul/LakeSoulExternalCatalog.java @@ -107,4 +107,3 @@ public List listPartitionInfo(String tableId) { return Lists.newArrayList(); } } - diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeCacheKey.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeCacheKey.java deleted file mode 100644 index 6e3d860a080e14..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeCacheKey.java +++ /dev/null @@ -1,65 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.datasource.maxcompute; - -import lombok.Data; - -import java.util.Objects; - -@Data -public class MaxComputeCacheKey { - private final String dbName; - private final String tblName; - private String partitionSpec; // optional - - public MaxComputeCacheKey(String dbName, String tblName) { - this(dbName, tblName, null); - } - - public MaxComputeCacheKey(String dbName, String tblName, String partitionSpec) { - this.dbName = dbName; - this.tblName = tblName; - this.partitionSpec = partitionSpec; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (!(obj instanceof MaxComputeCacheKey)) { - return false; - } - boolean partitionEquals = true; - if (partitionSpec != null) { - partitionEquals = partitionSpec.equals(((MaxComputeCacheKey) obj).partitionSpec); - } - return partitionEquals && dbName.equals(((MaxComputeCacheKey) obj).dbName) - && tblName.equals(((MaxComputeCacheKey) obj).tblName); - } - - @Override - public int hashCode() { - return Objects.hash(dbName, tblName); - } - - @Override - public String toString() { - return "TablePartitionKey{" + "dbName='" + dbName + '\'' + ", tblName='" + tblName + '\'' + '}'; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalMetaCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalMetaCache.java new file mode 100644 index 00000000000000..46e58f1e380081 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalMetaCache.java @@ -0,0 +1,115 @@ +// 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.datasource.maxcompute; + +import org.apache.doris.common.Config; +import org.apache.doris.datasource.CacheException; +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.SchemaCacheKey; +import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.doris.datasource.TablePartitionValues; +import org.apache.doris.datasource.metacache.AbstractExternalMetaCache; +import org.apache.doris.datasource.metacache.CacheSpec; +import org.apache.doris.datasource.metacache.MetaCacheEntryDef; +import org.apache.doris.datasource.metacache.MetaCacheEntryInvalidation; + +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.ExecutorService; + +/** + * MaxCompute engine implementation of {@link AbstractExternalMetaCache}. + * + *

Registered entries: + *

    + *
  • {@code partition_values}: partition value/index structures per table
  • + *
  • {@code schema}: schema cache keyed by {@link SchemaCacheKey}
  • + *
+ */ +public class MaxComputeExternalMetaCache extends AbstractExternalMetaCache { + public static final String ENGINE = "maxcompute"; + public static final String ENTRY_PARTITION_VALUES = "partition_values"; + public static final String ENTRY_SCHEMA = "schema"; + private final EntryHandle partitionValuesEntry; + private final EntryHandle schemaEntry; + + public MaxComputeExternalMetaCache(ExecutorService refreshExecutor) { + super(ENGINE, refreshExecutor); + partitionValuesEntry = registerEntry(MetaCacheEntryDef.contextualOnly( + ENTRY_PARTITION_VALUES, + NameMapping.class, + TablePartitionValues.class, + CacheSpec.of( + true, + Config.external_cache_refresh_time_minutes * 60L, + Config.max_hive_partition_cache_num), + MetaCacheEntryInvalidation.forNameMapping(nameMapping -> nameMapping))); + schemaEntry = registerEntry(MetaCacheEntryDef.of( + ENTRY_SCHEMA, + SchemaCacheKey.class, + SchemaCacheValue.class, + this::loadSchemaCacheValue, + defaultSchemaCacheSpec(), + MetaCacheEntryInvalidation.forNameMapping(SchemaCacheKey::getNameMapping))); + } + + @Override + public Collection aliases() { + return Collections.singleton("max_compute"); + } + + public TablePartitionValues getPartitionValues(NameMapping nameMapping) { + return partitionValuesEntry.get(nameMapping.getCtlId()).get(nameMapping, this::loadPartitionValues); + } + + public MaxComputeSchemaCacheValue getMaxComputeSchemaCacheValue(long catalogId, SchemaCacheKey key) { + SchemaCacheValue schemaCacheValue = schemaEntry.get(catalogId).get(key); + return (MaxComputeSchemaCacheValue) schemaCacheValue; + } + + private SchemaCacheValue loadSchemaCacheValue(SchemaCacheKey key) { + ExternalTable dorisTable = findExternalTable(key.getNameMapping(), ENGINE); + return dorisTable.initSchemaAndUpdateTime(key).orElseThrow(() -> + new CacheException("failed to load maxcompute schema cache value for: %s.%s.%s", + null, key.getNameMapping().getCtlId(), key.getNameMapping().getLocalDbName(), + key.getNameMapping().getLocalTblName())); + } + + private TablePartitionValues loadPartitionValues(NameMapping nameMapping) { + MaxComputeSchemaCacheValue schemaCacheValue = + getMaxComputeSchemaCacheValue(nameMapping.getCtlId(), new SchemaCacheKey(nameMapping)); + TablePartitionValues partitionValues = new TablePartitionValues(); + partitionValues.addPartitions( + schemaCacheValue.getPartitionSpecs(), + schemaCacheValue.getPartitionSpecs().stream() + .map(spec -> MaxComputeExternalTable.parsePartitionValues( + schemaCacheValue.getPartitionColumnNames(), spec)) + .collect(java.util.stream.Collectors.toList()), + schemaCacheValue.getPartitionTypes(), + Collections.nCopies(schemaCacheValue.getPartitionSpecs().size(), 0L)); + return partitionValues; + } + + @Override + protected Map catalogPropertyCompatibilityMap() { + return singleCompatibilityMap(ExternalCatalog.SCHEMA_CACHE_TTL_SECOND, ENTRY_SCHEMA); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java index d7724b1c7e0181..839995ca5f61e4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java @@ -66,6 +66,11 @@ public MaxComputeExternalTable(long id, String name, String remoteName, MaxCompu super(id, name, remoteName, catalog, db, TableType.MAX_COMPUTE_EXTERNAL_TABLE); } + @Override + public String getMetaCacheEngine() { + return MaxComputeExternalMetaCache.ENGINE; + } + @Override protected synchronized void makeSureInitialized() { super.makeSureInitialized(); @@ -114,27 +119,9 @@ private TablePartitionValues getPartitionValues() { if (!schemaCacheValue.isPresent()) { return new TablePartitionValues(); } - Table odpsTable = ((MaxComputeSchemaCacheValue) schemaCacheValue.get()).getOdpsTable(); - String projectName = odpsTable.getProject(); - String tableName = odpsTable.getName(); - MaxComputeMetadataCache metadataCache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getMaxComputeMetadataCache(catalog.getId()); - return metadataCache.getCachedPartitionValues( - new MaxComputeCacheKey(projectName, tableName), - key -> loadPartitionValues((MaxComputeSchemaCacheValue) schemaCacheValue.get())); - } - - private TablePartitionValues loadPartitionValues(MaxComputeSchemaCacheValue schemaCacheValue) { - List partitionSpecs = schemaCacheValue.getPartitionSpecs(); - List partitionTypes = schemaCacheValue.getPartitionTypes(); - List partitionColumnNames = schemaCacheValue.getPartitionColumnNames(); - TablePartitionValues partitionValues = new TablePartitionValues(); - partitionValues.addPartitions(partitionSpecs, - partitionSpecs.stream() - .map(p -> parsePartitionValues(partitionColumnNames, p)) - .collect(Collectors.toList()), - partitionTypes, Collections.nCopies(partitionSpecs.size(), 0L)); - return partitionValues; + MaxComputeExternalMetaCache metadataCache = Env.getCurrentEnv().getExtMetaCacheMgr() + .maxCompute(getCatalog().getId()); + return metadataCache.getPartitionValues(getOrBuildNameMapping()); } /** @@ -146,7 +133,7 @@ private TablePartitionValues loadPartitionValues(MaxComputeSchemaCacheValue sche * @param partitionPath partitionPath format is like the 'part1=123/part2=abc/part3=1bc' * @return all values of partitionPath */ - private static List parsePartitionValues(List partitionColumns, String partitionPath) { + static List parsePartitionValues(List partitionColumns, String partitionPath) { String[] partitionFragments = partitionPath.split("/"); if (partitionFragments.length != partitionColumns.size()) { throw new RuntimeException("Failed to parse partition values of path: " + partitionPath); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeMetadataCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeMetadataCache.java deleted file mode 100644 index f2a9b78fd9fe37..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeMetadataCache.java +++ /dev/null @@ -1,61 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.datasource.maxcompute; - -import org.apache.doris.common.Config; -import org.apache.doris.datasource.TablePartitionValues; - -import com.github.benmanes.caffeine.cache.Cache; -import com.github.benmanes.caffeine.cache.Caffeine; - -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.function.Function; -import java.util.stream.Collectors; - -public class MaxComputeMetadataCache { - private final Cache partitionValuesCache; - - public MaxComputeMetadataCache() { - partitionValuesCache = Caffeine.newBuilder().maximumSize(Config.max_hive_partition_cache_num) - .expireAfterAccess(Config.external_cache_refresh_time_minutes, TimeUnit.MINUTES) - .build(); - } - - public TablePartitionValues getCachedPartitionValues(MaxComputeCacheKey tablePartitionKey, - Function loader) { - return partitionValuesCache.get(tablePartitionKey, loader); - } - - public void cleanUp() { - partitionValuesCache.invalidateAll(); - } - - public void cleanDatabaseCache(String dbName) { - List removeCacheList = partitionValuesCache.asMap().keySet() - .stream() - .filter(k -> k.getDbName().equalsIgnoreCase(dbName)) - .collect(Collectors.toList()); - partitionValuesCache.invalidateAll(removeCacheList); - } - - public void cleanTableCache(String dbName, String tblName) { - MaxComputeCacheKey cacheKey = new MaxComputeCacheKey(dbName, tblName); - partitionValuesCache.invalidate(cacheKey); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeMetadataCacheMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeMetadataCacheMgr.java deleted file mode 100644 index c8b87519a78a05..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeMetadataCacheMgr.java +++ /dev/null @@ -1,69 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.datasource.maxcompute; - -import org.apache.doris.datasource.ExternalTable; - -import com.google.common.collect.Maps; - -import java.util.Map; - -public class MaxComputeMetadataCacheMgr { - - private final Map maxComputeMetadataCaches = Maps.newConcurrentMap(); - - public MaxComputeMetadataCacheMgr() { - } - - public MaxComputeMetadataCache getMaxComputeMetadataCache(long catalogId) { - MaxComputeMetadataCache cache = maxComputeMetadataCaches.get(catalogId); - if (cache == null) { - cache = new MaxComputeMetadataCache(); - maxComputeMetadataCaches.put(catalogId, cache); - } - return cache; - } - - public void removeCache(long catalogId) { - MaxComputeMetadataCache cache = maxComputeMetadataCaches.remove(catalogId); - if (cache != null) { - cache.cleanUp(); - } - } - - public void invalidateCatalogCache(long catalogId) { - MaxComputeMetadataCache cache = maxComputeMetadataCaches.get(catalogId); - if (cache != null) { - cache.cleanUp(); - } - } - - public void invalidateDbCache(long catalogId, String dbName) { - MaxComputeMetadataCache cache = maxComputeMetadataCaches.get(catalogId); - if (cache != null) { - cache.cleanDatabaseCache(dbName); - } - } - - public void invalidateTableCache(ExternalTable dorisTable) { - MaxComputeMetadataCache cache = maxComputeMetadataCaches.get(dorisTable.getCatalog().getId()); - if (cache != null) { - cache.cleanTableCache(dorisTable.getDbName(), dorisTable.getName()); - } - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/AbstractExternalMetaCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/AbstractExternalMetaCache.java new file mode 100644 index 00000000000000..a3a44151e45e2f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/AbstractExternalMetaCache.java @@ -0,0 +1,334 @@ +// 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.datasource.metacache; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.Config; +import org.apache.doris.datasource.CacheException; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.SchemaCacheValue; + +import com.google.common.collect.Maps; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ExecutorService; +import java.util.function.Function; +import java.util.function.Predicate; + +/** + * Base implementation of {@link ExternalMetaCache}. + * It keeps the shared in-memory layout + * Map<CatalogId, CatalogEntryGroup>, implements default + * lifecycle behavior, and provides conservative invalidation fallback. + * Subclasses register entry definitions during construction and expect callers + * to initialize a catalog explicitly before accessing entries. + */ +public abstract class AbstractExternalMetaCache implements ExternalMetaCache { + protected static CacheSpec defaultEntryCacheSpec() { + return CacheSpec.of( + true, + Config.external_cache_expire_time_seconds_after_access, + Config.max_external_table_cache_num); + } + + protected static CacheSpec defaultSchemaCacheSpec() { + return CacheSpec.of( + true, + Config.external_cache_expire_time_seconds_after_access, + Config.max_external_schema_cache_num); + } + + private final String engine; + private final ExecutorService refreshExecutor; + private final Map catalogEntries = Maps.newConcurrentMap(); + private final Map> metaCacheEntryDefs = Maps.newConcurrentMap(); + + protected AbstractExternalMetaCache(String engine, ExecutorService refreshExecutor) { + this.engine = engine; + this.refreshExecutor = Objects.requireNonNull(refreshExecutor, "refreshExecutor can not be null"); + } + + @Override + public String engine() { + return engine; + } + + @Override + public Collection aliases() { + return Collections.singleton(engine); + } + + @Override + public void initCatalog(long catalogId, Map catalogProperties) { + Map safeCatalogProperties = CacheSpec.applyCompatibilityMap( + catalogProperties, catalogPropertyCompatibilityMap()); + catalogEntries.computeIfAbsent(catalogId, id -> buildCatalogEntryGroup(safeCatalogProperties)); + } + + @Override + public void checkCatalogInitialized(long catalogId) { + requireCatalogEntryGroup(catalogId); + } + + @Override + public boolean isCatalogInitialized(long catalogId) { + return catalogEntries.containsKey(catalogId); + } + + /** + * Optional compatibility map for legacy catalog properties. + * + *

Map format: {@code legacyKey -> newKey}. The mapping is applied before + * entry cache specs are parsed. If both keys exist, new key keeps precedence. + */ + protected Map catalogPropertyCompatibilityMap() { + return Collections.emptyMap(); + } + + @Override + @SuppressWarnings("unchecked") + public MetaCacheEntry entry(long catalogId, String entryName, Class keyType, Class valueType) { + CatalogEntryGroup group = requireCatalogEntryGroup(catalogId); + MetaCacheEntryDef def = requireMetaCacheEntryDef(entryName); + ensureTypeCompatible(def, keyType, valueType); + + MetaCacheEntry cacheEntry = group.get(entryName); + if (cacheEntry == null) { + throw new IllegalStateException(String.format( + "Entry '%s' is not initialized for engine '%s', catalog %d.", + entryName, engine, catalogId)); + } + return (MetaCacheEntry) cacheEntry; + } + + @Override + public void invalidateCatalog(long catalogId) { + CatalogEntryGroup removed = catalogEntries.remove(catalogId); + if (removed != null) { + removed.invalidateAll(); + } + } + + @Override + public void invalidateCatalogEntries(long catalogId) { + CatalogEntryGroup group = catalogEntries.get(catalogId); + if (group != null) { + group.invalidateAll(); + } + } + + @Override + public void invalidateDb(long catalogId, String dbName) { + invalidateEntries(catalogId, entryDef -> entryDef.getInvalidation().dbPredicate(dbName)); + } + + @Override + public void invalidateTable(long catalogId, String dbName, String tableName) { + invalidateEntries(catalogId, entryDef -> entryDef.getInvalidation().tablePredicate(dbName, tableName)); + } + + @Override + public void invalidatePartitions(long catalogId, String dbName, String tableName, List partitions) { + invalidateEntries(catalogId, + entryDef -> entryDef.getInvalidation().partitionPredicate(dbName, tableName, partitions)); + } + + @Override + public Map stats(long catalogId) { + CatalogEntryGroup group = catalogEntries.get(catalogId); + return group == null ? Maps.newHashMap() : group.stats(); + } + + @Override + public void close() { + catalogEntries.values().forEach(CatalogEntryGroup::invalidateAll); + catalogEntries.clear(); + } + + protected final void registerMetaCacheEntryDef(MetaCacheEntryDef entryDef) { + Objects.requireNonNull(entryDef, "entryDef"); + if (!catalogEntries.isEmpty()) { + throw new IllegalStateException( + String.format("Can not register entry '%s' after catalog initialization for engine '%s'.", + entryDef.getName(), engine)); + } + MetaCacheEntryDef existing = metaCacheEntryDefs.putIfAbsent(entryDef.getName(), entryDef); + if (existing != null) { + throw new IllegalArgumentException( + String.format("Duplicated entry definition '%s' for engine '%s'.", entryDef.getName(), engine)); + } + } + + protected final EntryHandle registerEntry(MetaCacheEntryDef entryDef) { + registerMetaCacheEntryDef(entryDef); + return new EntryHandle<>(entryDef); + } + + protected final MetaCacheEntry entry(long catalogId, MetaCacheEntryDef entryDef) { + validateRegisteredMetaCacheEntryDef(entryDef); + return entry(catalogId, entryDef.getName(), entryDef.getKeyType(), entryDef.getValueType()); + } + + protected final String metaCacheTtlKey(String entryName) { + return "meta.cache." + engine + "." + entryName + ".ttl-second"; + } + + protected final Map singleCompatibilityMap(String legacyKey, String entryName) { + return Collections.singletonMap(legacyKey, metaCacheTtlKey(entryName)); + } + + protected final boolean matchDb(NameMapping nameMapping, String dbName) { + return nameMapping.getLocalDbName().equals(dbName); + } + + protected final boolean matchTable(NameMapping nameMapping, String dbName, String tableName) { + return matchDb(nameMapping, dbName) && nameMapping.getLocalTblName().equals(tableName); + } + + protected final ExternalTable findExternalTable(NameMapping nameMapping, String engineNameForError) { + CatalogIf catalog = getCatalog(nameMapping.getCtlId()); + if (!(catalog instanceof ExternalCatalog)) { + throw new CacheException("catalog %s is not external when loading %s schema cache", + null, nameMapping.getCtlId(), engineNameForError); + } + ExternalCatalog externalCatalog = (ExternalCatalog) catalog; + return externalCatalog.getDb(nameMapping.getLocalDbName()) + .flatMap(db -> db.getTable(nameMapping.getLocalTblName())) + .orElseThrow(() -> new CacheException( + "table %s.%s.%s not found when loading %s schema cache", + null, nameMapping.getCtlId(), nameMapping.getLocalDbName(), + nameMapping.getLocalTblName(), engineNameForError)); + } + + private CatalogEntryGroup requireCatalogEntryGroup(long catalogId) { + CatalogEntryGroup group = catalogEntries.get(catalogId); + if (group == null) { + throw new IllegalStateException(String.format( + "Catalog %d is not initialized for engine '%s'.", + catalogId, engine)); + } + return group; + } + + protected CatalogIf getCatalog(long catalogId) { + if (Env.getCurrentEnv() == null || Env.getCurrentEnv().getCatalogMgr() == null) { + return null; + } + return Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogId); + } + + private MetaCacheEntryDef requireMetaCacheEntryDef(String entryName) { + MetaCacheEntryDef entryDef = metaCacheEntryDefs.get(entryName); + if (entryDef == null) { + throw new IllegalArgumentException(String.format( + "Entry '%s' is not registered for engine '%s'.", entryName, engine)); + } + return entryDef; + } + + private void ensureTypeCompatible(MetaCacheEntryDef entryDef, Class keyType, Class valueType) { + if (!entryDef.getKeyType().equals(keyType) || !entryDef.getValueType().equals(valueType)) { + throw new IllegalArgumentException(String.format( + "Entry '%s' for engine '%s' expects key/value types (%s, %s), but got (%s, %s).", + entryDef.getName(), engine, entryDef.getKeyType().getName(), entryDef.getValueType().getName(), + keyType.getName(), valueType.getName())); + } + } + + private void validateRegisteredMetaCacheEntryDef(MetaCacheEntryDef entryDef) { + MetaCacheEntryDef registered = requireMetaCacheEntryDef(entryDef.getName()); + ensureTypeCompatible(registered, entryDef.getKeyType(), entryDef.getValueType()); + } + + private void invalidateEntries(long catalogId, Function, Predicate> predicateFactory) { + CatalogEntryGroup group = catalogEntries.get(catalogId); + if (group == null) { + return; + } + metaCacheEntryDefs.values().forEach(entryDef -> invalidateEntryIfMatched(group, entryDef, predicateFactory)); + } + + @SuppressWarnings("unchecked") + private void invalidateEntryIfMatched(CatalogEntryGroup group, MetaCacheEntryDef entryDef, + Function, Predicate> predicateFactory) { + Predicate predicate = (Predicate) predicateFactory.apply(entryDef); + if (predicate == null) { + return; + } + MetaCacheEntry entry = (MetaCacheEntry) group.get(entryDef.getName()); + if (entry != null) { + entry.invalidateIf(predicate); + } + } + + private CatalogEntryGroup buildCatalogEntryGroup(Map catalogProperties) { + CatalogEntryGroup group = new CatalogEntryGroup(); + metaCacheEntryDefs.values() + .forEach(entryDef -> group.put(entryDef.getName(), newMetaCacheEntry(entryDef, catalogProperties))); + return group; + } + + @SuppressWarnings("unchecked") + private MetaCacheEntry newMetaCacheEntry( + MetaCacheEntryDef rawEntryDef, Map catalogProperties) { + MetaCacheEntryDef entryDef = (MetaCacheEntryDef) rawEntryDef; + CacheSpec cacheSpec = CacheSpec.fromProperties( + catalogProperties, engine, entryDef.getName(), entryDef.getDefaultCacheSpec()); + return new MetaCacheEntry<>(entryDef.getName(), + wrapSchemaValidator(entryDef.getLoader(), entryDef.getValueType()), + cacheSpec, + refreshExecutor, entryDef.isAutoRefresh(), entryDef.isContextualOnly()); + } + + private Function wrapSchemaValidator(Function loader, Class valueType) { + if (loader == null) { + return null; + } + if (!SchemaCacheValue.class.isAssignableFrom(valueType)) { + return loader; + } + return key -> { + V value = loader.apply(key); + ((SchemaCacheValue) value).validateSchema(); + return value; + }; + } + + protected final class EntryHandle { + private final MetaCacheEntryDef entryDef; + + private EntryHandle(MetaCacheEntryDef entryDef) { + this.entryDef = entryDef; + } + + public MetaCacheEntry get(long catalogId) { + return entry(catalogId, entryDef); + } + + public MetaCacheEntry getIfInitialized(long catalogId) { + return isCatalogInitialized(catalogId) ? get(catalogId) : null; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/CacheSpec.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/CacheSpec.java index ca0f1be330d753..0bb640ad0d753c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/CacheSpec.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/CacheSpec.java @@ -21,7 +21,9 @@ import org.apache.commons.lang3.math.NumberUtils; +import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.OptionalLong; /** @@ -37,6 +39,10 @@ public final class CacheSpec { public static final long CACHE_NO_TTL = -1L; public static final long CACHE_TTL_DISABLE_CACHE = 0L; + private static final String META_CACHE_PREFIX = "meta.cache."; + private static final String KEY_ENABLE = ".enable"; + private static final String KEY_TTL_SECOND = ".ttl-second"; + private static final String KEY_CAPACITY = ".capacity"; private final boolean enable; private final long ttlSecond; @@ -48,39 +54,73 @@ private CacheSpec(boolean enable, long ttlSecond, long capacity) { this.capacity = capacity; } + public static CacheSpec of(boolean enable, long ttlSecond, long capacity) { + return new CacheSpec(enable, ttlSecond, capacity); + } + + public static PropertySpec.Builder propertySpecBuilder() { + return new PropertySpec.Builder(); + } + public static CacheSpec fromProperties(Map properties, String enableKey, boolean defaultEnable, String ttlKey, long defaultTtlSecond, String capacityKey, long defaultCapacity) { - boolean enable = getBooleanProperty(properties, enableKey, defaultEnable); - long ttlSecond = getLongProperty(properties, ttlKey, defaultTtlSecond); - long capacity = getLongProperty(properties, capacityKey, defaultCapacity); - if (!isCacheEnabled(enable, ttlSecond, capacity)) { - capacity = 0; - } - return new CacheSpec(enable, ttlSecond, capacity); + return fromProperties(properties, propertySpecBuilder() + .enable(enableKey, defaultEnable) + .ttl(ttlKey, defaultTtlSecond) + .capacity(capacityKey, defaultCapacity) + .build()); + } + + public static CacheSpec fromProperties(Map properties, PropertySpec propertySpec) { + boolean enable = getBooleanProperty(properties, propertySpec.getEnableKey(), propertySpec.isDefaultEnable()); + long ttlSecond = getLongProperty(properties, propertySpec.getTtlKey(), propertySpec.getDefaultTtlSecond()); + long capacity = getLongProperty(properties, propertySpec.getCapacityKey(), propertySpec.getDefaultCapacity()); + return of(enable, ttlSecond, capacity); + } + + /** + * Build a cache spec from catalog properties by standard external meta cache key pattern: + * meta.cache.<engine>.<entry>.(enable|ttl-second|capacity) + */ + public static CacheSpec fromProperties(Map properties, + String engine, String entryName, CacheSpec defaultSpec) { + return fromProperties(properties, metaCachePropertySpec(engine, entryName, defaultSpec)); + } + + public static PropertySpec metaCachePropertySpec(String engine, String entryName, CacheSpec defaultSpec) { + String cacheKeyPrefix = META_CACHE_PREFIX + engine + "." + entryName; + return propertySpecBuilder() + .enable(cacheKeyPrefix + KEY_ENABLE, defaultSpec.isEnable()) + .ttl(cacheKeyPrefix + KEY_TTL_SECOND, defaultSpec.getTtlSecond()) + .capacity(cacheKeyPrefix + KEY_CAPACITY, defaultSpec.getCapacity()) + .build(); } /** - * Build a cache spec from a ttl property value and fixed capacity. + * Apply compatibility key mapping before cache spec parsing. * - *

Semantics are compatible with legacy schema cache behavior: - *

    - *
  • ttlValue is null: use default ttl
  • - *
  • ttl=-1: no expiration
  • - *
  • ttl=0: disable cache by forcing capacity=0
  • - *
  • ttl parse failure: fallback to -1 (no expiration)
  • - *
- * TODO: Refactor schema cache and its parameters to the unified enable/ttl/capacity model, - * then remove this ttl-only adapter. + *

Map format: {@code legacyKey -> newKey}. If both keys exist, new key wins. */ - public static CacheSpec fromTtlValue(String ttlValue, long defaultTtlSecond, long defaultCapacity) { - long ttlSecond = ttlValue == null ? defaultTtlSecond : NumberUtils.toLong(ttlValue, CACHE_NO_TTL); - long capacity = defaultCapacity; - if (!isCacheEnabled(true, ttlSecond, capacity)) { - capacity = 0; + public static Map applyCompatibilityMap( + Map properties, Map compatibilityMap) { + Map mapped = new HashMap<>(); + if (properties != null) { + mapped.putAll(properties); + } + if (compatibilityMap == null || compatibilityMap.isEmpty()) { + return mapped; } - return new CacheSpec(true, ttlSecond, capacity); + compatibilityMap.forEach((legacyKey, newKey) -> { + if (legacyKey == null || newKey == null || legacyKey.equals(newKey)) { + return; + } + if (!mapped.containsKey(newKey) && mapped.containsKey(legacyKey)) { + mapped.put(newKey, mapped.get(legacyKey)); + } + }); + return mapped; } public static void checkBooleanProperty(String value, String key) throws DdlException { @@ -111,6 +151,21 @@ public static boolean isCacheEnabled(boolean enable, long ttlSecond, long capaci return enable && ttlSecond != 0 && capacity != 0; } + /** + * Build standard external meta cache key prefix for one engine. + * Example: {@code meta.cache.iceberg.} + */ + public static String metaCacheKeyPrefix(String engine) { + return META_CACHE_PREFIX + engine + "."; + } + + /** + * Returns true when the given property key belongs to one engine's meta cache namespace. + */ + public static boolean isMetaCacheKeyForEngine(String key, String engine) { + return key != null && engine != null && key.startsWith(metaCacheKeyPrefix(engine)); + } + /** * Convert ttlSecond to OptionalLong for CacheFactory. * ttlSecond=-1 means no expiration; ttlSecond=0 disables cache. @@ -149,4 +204,84 @@ public long getTtlSecond() { public long getCapacity() { return capacity; } + + public static final class PropertySpec { + private final String enableKey; + private final boolean defaultEnable; + private final String ttlKey; + private final long defaultTtlSecond; + private final String capacityKey; + private final long defaultCapacity; + + private PropertySpec(String enableKey, boolean defaultEnable, String ttlKey, + long defaultTtlSecond, String capacityKey, long defaultCapacity) { + this.enableKey = enableKey; + this.defaultEnable = defaultEnable; + this.ttlKey = ttlKey; + this.defaultTtlSecond = defaultTtlSecond; + this.capacityKey = capacityKey; + this.defaultCapacity = defaultCapacity; + } + + public String getEnableKey() { + return enableKey; + } + + public boolean isDefaultEnable() { + return defaultEnable; + } + + public String getTtlKey() { + return ttlKey; + } + + public long getDefaultTtlSecond() { + return defaultTtlSecond; + } + + public String getCapacityKey() { + return capacityKey; + } + + public long getDefaultCapacity() { + return defaultCapacity; + } + + public static final class Builder { + private String enableKey; + private boolean defaultEnable; + private String ttlKey; + private long defaultTtlSecond; + private String capacityKey; + private long defaultCapacity; + + public Builder enable(String key, boolean defaultValue) { + this.enableKey = key; + this.defaultEnable = defaultValue; + return this; + } + + public Builder ttl(String key, long defaultValue) { + this.ttlKey = key; + this.defaultTtlSecond = defaultValue; + return this; + } + + public Builder capacity(String key, long defaultValue) { + this.capacityKey = key; + this.defaultCapacity = defaultValue; + return this; + } + + public PropertySpec build() { + return new PropertySpec( + Objects.requireNonNull(enableKey, "enableKey is required"), + defaultEnable, + Objects.requireNonNull(ttlKey, "ttlKey is required"), + defaultTtlSecond, + Objects.requireNonNull(capacityKey, "capacityKey is required"), + defaultCapacity); + } + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogScopedCacheMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/CatalogEntryGroup.java similarity index 50% rename from fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogScopedCacheMgr.java rename to fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/CatalogEntryGroup.java index 6b9afc1b74a6de..c195087f415bfc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogScopedCacheMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/CatalogEntryGroup.java @@ -15,29 +15,35 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.datasource; +package org.apache.doris.datasource.metacache; + +import com.google.common.collect.Maps; import java.util.Map; +import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; -import java.util.function.Function; -public class CatalogScopedCacheMgr { - private final Map cacheMap = new ConcurrentHashMap<>(); - private final Function cacheFactory; +/** + * Catalog scoped entry container. + */ +public class CatalogEntryGroup { + private final Map> entries = new ConcurrentHashMap<>(); - public CatalogScopedCacheMgr(Function cacheFactory) { - this.cacheFactory = cacheFactory; + public MetaCacheEntry get(String entryName) { + return entries.get(entryName); } - public T getCache(ExternalCatalog catalog) { - return cacheMap.computeIfAbsent(catalog.getId(), id -> cacheFactory.apply(catalog)); + public void put(String entryName, MetaCacheEntry entry) { + entries.put(Objects.requireNonNull(entryName, "entryName"), Objects.requireNonNull(entry, "entry")); } - public T getCache(long catalogId) { - return cacheMap.get(catalogId); + public Map stats() { + Map result = Maps.newHashMap(); + entries.forEach((name, entry) -> result.put(name, entry.stats())); + return result; } - public T removeCache(long catalogId) { - return cacheMap.remove(catalogId); + public void invalidateAll() { + entries.values().forEach(MetaCacheEntry::invalidateAll); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/ExternalMetaCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/ExternalMetaCache.java new file mode 100644 index 00000000000000..1a067726ec9136 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/ExternalMetaCache.java @@ -0,0 +1,120 @@ +// 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.datasource.metacache; + +import org.apache.doris.datasource.SchemaCacheKey; +import org.apache.doris.datasource.SchemaCacheValue; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** + * Engine-level abstraction for external metadata cache. + * It defines a unified access path (engine -> catalog -> entry), scoped + * invalidation APIs, and a common stats output shape. + */ +public interface ExternalMetaCache { + /** + * @return engine type name, for example: hive/iceberg/paimon. + */ + String engine(); + + /** + * Additional engine aliases accepted by the manager. + */ + Collection aliases(); + + /** + * Initialize all registered entries for one catalog under current engine. + * Entry instances are created eagerly at this stage. + * + * @param catalogId catalog id + * @param catalogProperties catalog-level properties used for resolving + * entry cache configuration + */ + void initCatalog(long catalogId, Map catalogProperties); + + /** + * Get one cache entry under an engine and catalog. + * + *

This is a low-level extension API. Prefer typed engine operations when + * available. + */ + MetaCacheEntry entry(long catalogId, String entryName, Class keyType, Class valueType); + + /** + * Validate that the catalog has been initialized in current engine cache. + */ + void checkCatalogInitialized(long catalogId); + + /** + * Returns whether the catalog is currently initialized in this engine cache. + */ + boolean isCatalogInitialized(long catalogId); + + /** + * Typed schema cache access that hides entry-name and class plumbing from callers. + */ + @SuppressWarnings("unchecked") + default Optional getSchemaValue(long catalogId, K key) { + return Optional.ofNullable(entry(catalogId, "schema", (Class) key.getClass(), SchemaCacheValue.class) + .get(key)); + } + + /** + * Invalidate all entries under one catalog in current engine cache. + */ + void invalidateCatalog(long catalogId); + + /** + * Invalidate cached data under one catalog but keep the catalog entry group initialized. + * This is used by refresh flows where catalog lifecycle remains initialized. + * + *

The default implementation falls back to full catalog invalidation. + */ + default void invalidateCatalogEntries(long catalogId) { + invalidateCatalog(catalogId); + } + + /** + * Invalidate all entries related to a database. + */ + void invalidateDb(long catalogId, String dbName); + + /** + * Invalidate all entries related to a table. + */ + void invalidateTable(long catalogId, String dbName, String tableName); + + /** + * Invalidate all entries related to specific partitions. + */ + void invalidatePartitions(long catalogId, String dbName, String tableName, List partitions); + + /** + * Return stats of all entries under one catalog. + */ + Map stats(long catalogId); + + /** + * Release resources owned by current engine cache. + */ + void close(); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/ExternalMetaCacheRegistry.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/ExternalMetaCacheRegistry.java new file mode 100644 index 00000000000000..56a4753b3020b9 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/ExternalMetaCacheRegistry.java @@ -0,0 +1,99 @@ +// 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.datasource.metacache; + +import com.google.common.collect.Maps; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Collection; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; + +/** + * Registry for engine cache instances and alias resolution. + */ +public class ExternalMetaCacheRegistry { + private static final Logger LOG = LogManager.getLogger(ExternalMetaCacheRegistry.class); + private static final String ENGINE_DEFAULT = "default"; + + private final Map engineCaches = Maps.newConcurrentMap(); + private final Map engineAliasIndex = Maps.newConcurrentMap(); + + public ExternalMetaCache resolve(String engine) { + Objects.requireNonNull(engine, "engine is null"); + String normalizedEngine = normalizeEngineName(engine); + String primaryEngine = engineAliasIndex.getOrDefault(normalizedEngine, normalizedEngine); + ExternalMetaCache found = engineCaches.get(primaryEngine); + if (found != null) { + return found; + } + throw new IllegalArgumentException( + String.format("unsupported external meta cache engine '%s'", normalizedEngine)); + } + + public Collection allCaches() { + return engineCaches.values(); + } + + public void register(ExternalMetaCache cache) { + String engineName = normalizeEngineName(cache.engine()); + ExternalMetaCache existing = engineCaches.putIfAbsent(engineName, cache); + if (existing != null) { + LOG.warn("skip duplicated external meta cache engine '{}', existing class: {}, new class: {}", + engineName, existing.getClass().getName(), cache.getClass().getName()); + return; + } + registerAliases(cache, engineName); + LOG.debug("registered external meta cache engine '{}'", engineName); + } + + public void resetForTest(Collection caches) { + engineCaches.clear(); + engineAliasIndex.clear(); + caches.forEach(this::register); + } + + static String normalizeEngineName(String engine) { + if (engine == null) { + return ENGINE_DEFAULT; + } + String normalized = engine.trim().toLowerCase(Locale.ROOT); + if (normalized.isEmpty()) { + return ENGINE_DEFAULT; + } + return normalized; + } + + private void registerAliases(ExternalMetaCache cache, String primaryEngineName) { + registerAlias(primaryEngineName, primaryEngineName); + for (String alias : cache.aliases()) { + registerAlias(alias, primaryEngineName); + } + } + + private void registerAlias(String alias, String primaryEngineName) { + String normalizedAlias = normalizeEngineName(alias); + String existing = engineAliasIndex.putIfAbsent(normalizedAlias, primaryEngineName); + if (existing != null && !existing.equals(primaryEngineName)) { + LOG.warn("skip duplicated external meta cache alias '{}', existing engine: {}, new engine: {}", + normalizedAlias, existing, primaryEngineName); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/ExternalMetaCacheRouteResolver.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/ExternalMetaCacheRouteResolver.java new file mode 100644 index 00000000000000..48bde1ab99311f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/ExternalMetaCacheRouteResolver.java @@ -0,0 +1,93 @@ +// 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.datasource.metacache; + +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.doris.RemoteDorisExternalCatalog; +import org.apache.doris.datasource.hive.HMSExternalCatalog; +import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; +import org.apache.doris.datasource.maxcompute.MaxComputeExternalCatalog; +import org.apache.doris.datasource.paimon.PaimonExternalCatalog; + +import java.util.ArrayList; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Set; +import javax.annotation.Nullable; + +/** + * Resolves which engine caches should participate for a catalog lifecycle event. + */ +public class ExternalMetaCacheRouteResolver { + private static final String ENGINE_DEFAULT = "default"; + private static final String ENGINE_HIVE = "hive"; + private static final String ENGINE_HUDI = "hudi"; + private static final String ENGINE_ICEBERG = "iceberg"; + private static final String ENGINE_PAIMON = "paimon"; + private static final String ENGINE_MAXCOMPUTE = "maxcompute"; + private static final String ENGINE_DORIS = "doris"; + + private final ExternalMetaCacheRegistry registry; + + public ExternalMetaCacheRouteResolver(ExternalMetaCacheRegistry registry) { + this.registry = registry; + } + + public List resolveCatalogCaches(long catalogId, @Nullable CatalogIf catalog) { + Set resolved = new LinkedHashSet<>(); + if (catalog != null) { + addBuiltinRoutes(resolved, catalog); + return new ArrayList<>(resolved); + } + registry.allCaches().forEach(cache -> { + if (cache.isCatalogInitialized(catalogId)) { + resolved.add(cache); + } + }); + return new ArrayList<>(resolved); + } + + private void addBuiltinRoutes(Set resolved, CatalogIf catalog) { + if (catalog instanceof IcebergExternalCatalog) { + resolved.add(registry.resolve(ENGINE_ICEBERG)); + return; + } + if (catalog instanceof PaimonExternalCatalog) { + resolved.add(registry.resolve(ENGINE_PAIMON)); + return; + } + if (catalog instanceof MaxComputeExternalCatalog) { + resolved.add(registry.resolve(ENGINE_MAXCOMPUTE)); + return; + } + if (catalog instanceof RemoteDorisExternalCatalog) { + resolved.add(registry.resolve(ENGINE_DORIS)); + return; + } + if (catalog instanceof HMSExternalCatalog) { + resolved.add(registry.resolve(ENGINE_HIVE)); + resolved.add(registry.resolve(ENGINE_HUDI)); + resolved.add(registry.resolve(ENGINE_ICEBERG)); + return; + } + if (catalog instanceof ExternalCatalog) { + resolved.add(registry.resolve(ENGINE_DEFAULT)); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/LegacyMetaCacheFactory.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/LegacyMetaCacheFactory.java new file mode 100644 index 00000000000000..238cfdab951d7d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/LegacyMetaCacheFactory.java @@ -0,0 +1,49 @@ +// 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.datasource.metacache; + +import org.apache.doris.common.Pair; + +import com.github.benmanes.caffeine.cache.CacheLoader; +import com.github.benmanes.caffeine.cache.RemovalListener; + +import java.util.List; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.concurrent.ExecutorService; + +/** + * Bridge factory for legacy {@link MetaCache} users. + */ +public class LegacyMetaCacheFactory { + private final ExecutorService refreshExecutor; + + public LegacyMetaCacheFactory(ExecutorService refreshExecutor) { + this.refreshExecutor = refreshExecutor; + } + + public MetaCache build(String name, + OptionalLong expireAfterAccessSec, OptionalLong refreshAfterWriteSec, long maxSize, + CacheLoader>> namesCacheLoader, + CacheLoader> metaObjCacheLoader, + RemovalListener> removalListener) { + return new MetaCache<>( + name, refreshExecutor, expireAfterAccessSec, refreshAfterWriteSec, + maxSize, namesCacheLoader, metaObjCacheLoader, removalListener); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCacheEntry.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCacheEntry.java new file mode 100644 index 00000000000000..8913cd8f4ca000 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCacheEntry.java @@ -0,0 +1,188 @@ +// 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.datasource.metacache; + +import org.apache.doris.common.CacheFactory; +import org.apache.doris.common.Config; + +import com.github.benmanes.caffeine.cache.LoadingCache; +import com.github.benmanes.caffeine.cache.stats.CacheStats; + +import java.util.Objects; +import java.util.OptionalLong; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; +import java.util.function.Function; +import java.util.function.Predicate; +import javax.annotation.Nullable; + +/** + * Unified cache entry abstraction. + * It stores one logical cache dataset and provides optional lazy loading, + * key/predicate/full invalidation, and lightweight runtime stats. + */ +public class MetaCacheEntry { + private final String name; + @Nullable + private final Function loader; + private final CacheSpec cacheSpec; + private final boolean effectiveEnabled; + private final boolean autoRefresh; + private final LoadingCache data; + private final AtomicLong invalidateCount = new AtomicLong(0); + private final AtomicLong lastLoadSuccessTimeMs = new AtomicLong(-1L); + private final AtomicLong lastLoadFailureTimeMs = new AtomicLong(-1L); + private final AtomicReference lastError = new AtomicReference<>(""); + + public MetaCacheEntry(String name, Function loader, CacheSpec cacheSpec, ExecutorService refreshExecutor) { + this(name, loader, cacheSpec, refreshExecutor, true, false); + } + + public MetaCacheEntry(String name, Function loader, CacheSpec cacheSpec, ExecutorService refreshExecutor, + boolean autoRefresh) { + this(name, loader, cacheSpec, refreshExecutor, autoRefresh, false); + } + + public MetaCacheEntry(String name, @Nullable Function loader, CacheSpec cacheSpec, + ExecutorService refreshExecutor, boolean autoRefresh, boolean contextualOnly) { + this.name = name; + if (contextualOnly) { + if (loader != null) { + throw new IllegalArgumentException("contextual-only entry loader must be null"); + } + if (autoRefresh) { + throw new IllegalArgumentException("contextual-only entry can not enable auto refresh"); + } + } else { + Objects.requireNonNull(loader, "loader can not be null"); + } + this.loader = loader; + this.cacheSpec = Objects.requireNonNull(cacheSpec, "cacheSpec can not be null"); + this.autoRefresh = autoRefresh; + Objects.requireNonNull(refreshExecutor, "refreshExecutor can not be null"); + this.effectiveEnabled = CacheSpec.isCacheEnabled( + this.cacheSpec.isEnable(), this.cacheSpec.getTtlSecond(), this.cacheSpec.getCapacity()); + OptionalLong expireAfterAccessSec = + effectiveEnabled ? CacheSpec.toExpireAfterAccess(this.cacheSpec.getTtlSecond()) : OptionalLong.empty(); + OptionalLong refreshAfterWriteSec = + effectiveEnabled && autoRefresh + ? OptionalLong.of(Config.external_cache_refresh_time_minutes * 60) + : OptionalLong.empty(); + long maxSize = effectiveEnabled ? this.cacheSpec.getCapacity() : 0L; + CacheFactory cacheFactory = new CacheFactory( + expireAfterAccessSec, + refreshAfterWriteSec, + maxSize, + true, + null); + this.data = cacheFactory.buildCache(this::loadFromDefaultLoader, refreshExecutor); + } + + public String name() { + return name; + } + + public V get(K key) { + return data.get(key); + } + + public V get(K key, Function missLoader) { + Function loadFunction = Objects.requireNonNull(missLoader, "missLoader can not be null"); + return data.get(key, typedKey -> loadAndTrack(typedKey, loadFunction)); + } + + public V getIfPresent(K key) { + return data.getIfPresent(key); + } + + public void put(K key, V value) { + data.put(key, value); + } + + public void invalidateKey(K key) { + if (data.asMap().remove(key) != null) { + invalidateCount.incrementAndGet(); + } + } + + public void invalidateIf(Predicate predicate) { + data.asMap().keySet().removeIf(key -> { + if (predicate.test(key)) { + invalidateCount.incrementAndGet(); + return true; + } + return false; + }); + } + + public void invalidateAll() { + long size = data.estimatedSize(); + data.invalidateAll(); + invalidateCount.addAndGet(size); + } + + public void forEach(BiConsumer consumer) { + data.asMap().forEach(consumer); + } + + public MetaCacheEntryStats stats() { + CacheStats cacheStats = data.stats(); + return new MetaCacheEntryStats( + cacheSpec.isEnable(), + effectiveEnabled, + autoRefresh, + cacheSpec.getTtlSecond(), + cacheSpec.getCapacity(), + data.estimatedSize(), + cacheStats.requestCount(), + cacheStats.hitCount(), + cacheStats.missCount(), + cacheStats.hitRate(), + cacheStats.loadSuccessCount(), + cacheStats.loadFailureCount(), + cacheStats.totalLoadTime(), + cacheStats.averageLoadPenalty(), + cacheStats.evictionCount(), + invalidateCount.get(), + lastLoadSuccessTimeMs.get(), + lastLoadFailureTimeMs.get(), + lastError.get()); + } + + private V loadFromDefaultLoader(K key) { + if (loader == null) { + throw new UnsupportedOperationException( + String.format("Entry '%s' requires a contextual miss loader.", name)); + } + return loadAndTrack(key, loader); + } + + private V loadAndTrack(K key, Function loadFunction) { + try { + V value = loadFunction.apply(key); + lastLoadSuccessTimeMs.set(System.currentTimeMillis()); + return value; + } catch (RuntimeException | Error e) { + lastLoadFailureTimeMs.set(System.currentTimeMillis()); + lastError.set(e.toString()); + throw e; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCacheEntryDef.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCacheEntryDef.java new file mode 100644 index 00000000000000..1f48057a44fc40 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCacheEntryDef.java @@ -0,0 +1,235 @@ +// 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.datasource.metacache; + +import java.util.Objects; +import java.util.function.Function; +import javax.annotation.Nullable; + +/** + * Immutable definition of a logical {@link MetaCacheEntry}. + * + *

This class only describes "what an entry is", not "entry runtime data". + * Runtime instances are still created by {@link AbstractExternalMetaCache} + * per catalog during {@code initCatalog(long)}. + * + *

A definition contains: + *

    + *
  • a stable logical name ({@link #name})
  • + *
  • declared key/value Java types ({@link #keyType}/{@link #valueType})
  • + *
  • a required miss loader ({@link #loader})
  • + *
  • default cache spec ({@link #defaultCacheSpec}) used when catalog params are absent
  • + *
  • whether refresh-after-write is enabled ({@link #autoRefresh})
  • + *
+ * + *

Use case 1: load-on-miss entry (recommended for table/object metadata). + *

{@code
+ * private final MetaCacheEntryDef tableEntryDef =
+ *         MetaCacheEntryDef.of(
+ *                 "table",
+ *                 NameMapping.class,
+ *                 IcebergTableCacheValue.class,
+ *                 this::loadTableCacheValue,
+ *                 defaultEntryCacheSpec());
+ *
+ * registerMetaCacheEntryDef(tableEntryDef);
+ * }
+ */ +public final class MetaCacheEntryDef { + /** + * Logical entry name inside one engine. + * + *

It is used as: + *

    + *
  • the lookup key in catalog entry groups
  • + *
  • the identity in stats output and error messages
  • + *
+ * + *

Constraint: must be unique inside one concrete external meta cache + * implementation (for example inside one IcebergExternalMetaCache). + */ + private final String name; + + /** + * Declared key type of this entry. + * + *

Used by {@link AbstractExternalMetaCache} to validate that callers use + * the expected key class when obtaining the entry via + * {@code entry(catalogId, def)}. + */ + private final Class keyType; + + /** + * Declared value type of this entry. + * + *

Used by {@link AbstractExternalMetaCache} to validate value type + * compatibility for the requested entry and to prevent cross-entry misuse. + */ + private final Class valueType; + + /** + * Loader function used by {@link MetaCacheEntry#get(Object)}. + * + *

Cache miss triggers loader invocation. Loader is bound once at + * definition creation time and reused by all per-catalog runtime entries. + */ + @Nullable + private final Function loader; + + /** + * Default cache spec of this entry definition. + * + *

This value is used as fallback when no catalog-level override is provided. + * Keeping it on the definition makes each entry's default policy explicit. + */ + private final CacheSpec defaultCacheSpec; + private final boolean autoRefresh; + private final boolean contextualOnly; + private final MetaCacheEntryInvalidation invalidation; + + private MetaCacheEntryDef(String name, Class keyType, Class valueType, + @Nullable Function loader, CacheSpec defaultCacheSpec, boolean autoRefresh, boolean contextualOnly, + MetaCacheEntryInvalidation invalidation) { + this.name = Objects.requireNonNull(name, "entry name is required"); + this.keyType = Objects.requireNonNull(keyType, "entry key type is required"); + this.valueType = Objects.requireNonNull(valueType, "entry value type is required"); + if (contextualOnly) { + if (loader != null) { + throw new IllegalArgumentException("contextual-only entry loader must be null"); + } + if (autoRefresh) { + throw new IllegalArgumentException("contextual-only entry can not enable auto refresh"); + } + } else { + Objects.requireNonNull(loader, "entry loader is required"); + } + this.loader = loader; + this.defaultCacheSpec = Objects.requireNonNull(defaultCacheSpec, "entry default cache spec is required"); + this.autoRefresh = autoRefresh; + this.contextualOnly = contextualOnly; + this.invalidation = Objects.requireNonNull(invalidation, "entry invalidation is required"); + } + + /** + * Create an entry definition with miss loader and an explicit default cache spec. + * + * @param name logical entry name, unique in one cache implementation + * @param keyType declared key class + * @param valueType declared value class + * @param loader miss loader invoked by {@link MetaCacheEntry#get(Object)} + * @param defaultCacheSpec default cache policy used by this entry definition + */ + public static MetaCacheEntryDef of(String name, Class keyType, Class valueType, + Function loader, CacheSpec defaultCacheSpec) { + return of(name, keyType, valueType, loader, defaultCacheSpec, MetaCacheEntryInvalidation.none()); + } + + public static MetaCacheEntryDef of(String name, Class keyType, Class valueType, + Function loader, CacheSpec defaultCacheSpec, MetaCacheEntryInvalidation invalidation) { + return new MetaCacheEntryDef<>(name, keyType, valueType, loader, defaultCacheSpec, true, false, + invalidation); + } + + /** + * Create an entry definition with miss loader, explicit default cache spec and refresh policy. + * + * @param name logical entry name, unique in one cache implementation + * @param keyType declared key class + * @param valueType declared value class + * @param loader miss loader invoked by {@link MetaCacheEntry#get(Object)} + * @param defaultCacheSpec default cache policy used by this entry definition + * @param autoRefresh whether to enable refresh-after-write + */ + public static MetaCacheEntryDef of(String name, Class keyType, Class valueType, + Function loader, CacheSpec defaultCacheSpec, boolean autoRefresh) { + return of(name, keyType, valueType, loader, defaultCacheSpec, autoRefresh, MetaCacheEntryInvalidation.none()); + } + + public static MetaCacheEntryDef of(String name, Class keyType, Class valueType, + Function loader, CacheSpec defaultCacheSpec, boolean autoRefresh, + MetaCacheEntryInvalidation invalidation) { + return new MetaCacheEntryDef<>(name, keyType, valueType, loader, defaultCacheSpec, autoRefresh, false, + invalidation); + } + + /** + * Create an entry definition that requires callers to provide a contextual miss loader. + */ + public static MetaCacheEntryDef contextualOnly( + String name, Class keyType, Class valueType, CacheSpec defaultCacheSpec) { + return contextualOnly(name, keyType, valueType, defaultCacheSpec, MetaCacheEntryInvalidation.none()); + } + + public static MetaCacheEntryDef contextualOnly( + String name, Class keyType, Class valueType, CacheSpec defaultCacheSpec, + MetaCacheEntryInvalidation invalidation) { + return new MetaCacheEntryDef<>(name, keyType, valueType, null, defaultCacheSpec, false, true, + invalidation); + } + + /** + * @return logical entry name. + */ + public String getName() { + return name; + } + + /** + * @return declared key class. + */ + public Class getKeyType() { + return keyType; + } + + /** + * @return declared value class. + */ + public Class getValueType() { + return valueType; + } + + /** + * @return loader function. + */ + @Nullable + public Function getLoader() { + return loader; + } + + /** + * @return default cache spec of this entry definition. + */ + public CacheSpec getDefaultCacheSpec() { + return defaultCacheSpec; + } + + /** + * @return true when refresh-after-write is enabled. + */ + public boolean isAutoRefresh() { + return autoRefresh; + } + + public boolean isContextualOnly() { + return contextualOnly; + } + + public MetaCacheEntryInvalidation getInvalidation() { + return invalidation; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCacheEntryInvalidation.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCacheEntryInvalidation.java new file mode 100644 index 00000000000000..debd9a0df97763 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCacheEntryInvalidation.java @@ -0,0 +1,96 @@ +// 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.datasource.metacache; + +import org.apache.doris.datasource.NameMapping; + +import java.util.List; +import java.util.Objects; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.function.Predicate; +import javax.annotation.Nullable; + +/** + * Entry-level invalidation metadata used by {@link AbstractExternalMetaCache}. + */ +public final class MetaCacheEntryInvalidation { + @FunctionalInterface + public interface PartitionPredicateFactory { + Predicate create(String dbName, String tableName, List partitions); + } + + private static final MetaCacheEntryInvalidation NONE = new MetaCacheEntryInvalidation<>(null, null, null); + + @Nullable + private final Function> dbPredicateFactory; + @Nullable + private final BiFunction> tablePredicateFactory; + @Nullable + private final PartitionPredicateFactory partitionPredicateFactory; + + private MetaCacheEntryInvalidation( + @Nullable Function> dbPredicateFactory, + @Nullable BiFunction> tablePredicateFactory, + @Nullable PartitionPredicateFactory partitionPredicateFactory) { + this.dbPredicateFactory = dbPredicateFactory; + this.tablePredicateFactory = tablePredicateFactory; + this.partitionPredicateFactory = partitionPredicateFactory; + } + + @SuppressWarnings("unchecked") + public static MetaCacheEntryInvalidation none() { + return (MetaCacheEntryInvalidation) NONE; + } + + public static MetaCacheEntryInvalidation forNameMapping(Function nameMappingExtractor) { + Objects.requireNonNull(nameMappingExtractor, "nameMappingExtractor"); + return forTableIdentity( + key -> nameMappingExtractor.apply(key).getLocalDbName(), + key -> nameMappingExtractor.apply(key).getLocalTblName()); + } + + public static MetaCacheEntryInvalidation forTableIdentity( + Function dbNameExtractor, Function tableNameExtractor) { + Objects.requireNonNull(dbNameExtractor, "dbNameExtractor"); + Objects.requireNonNull(tableNameExtractor, "tableNameExtractor"); + return new MetaCacheEntryInvalidation<>( + dbName -> key -> dbNameExtractor.apply(key).equals(dbName), + (dbName, tableName) -> key -> dbNameExtractor.apply(key).equals(dbName) + && tableNameExtractor.apply(key).equals(tableName), + (dbName, tableName, partitions) -> key -> dbNameExtractor.apply(key).equals(dbName) + && tableNameExtractor.apply(key).equals(tableName)); + } + + @Nullable + Predicate dbPredicate(String dbName) { + return dbPredicateFactory == null ? null : dbPredicateFactory.apply(dbName); + } + + @Nullable + Predicate tablePredicate(String dbName, String tableName) { + return tablePredicateFactory == null ? null : tablePredicateFactory.apply(dbName, tableName); + } + + @Nullable + Predicate partitionPredicate(String dbName, String tableName, List partitions) { + return partitionPredicateFactory == null + ? null + : partitionPredicateFactory.create(dbName, tableName, partitions); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCacheEntryStats.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCacheEntryStats.java new file mode 100644 index 00000000000000..495fd011083bb0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/MetaCacheEntryStats.java @@ -0,0 +1,189 @@ +// 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.datasource.metacache; + +import java.util.Objects; + +/** + * Immutable stats snapshot of one {@link MetaCacheEntry}. + * + *

Time fields use the following units: + *

    + *
  • {@code totalLoadTimeNanos}/{@code averageLoadPenaltyNanos}: nanoseconds
  • + *
  • {@code lastLoadSuccessTimeMs}/{@code lastLoadFailureTimeMs}: epoch milliseconds
  • + *
+ * + *

For last-load timestamps, {@code -1} means no corresponding event happened yet. + * {@code lastError} keeps the latest load failure message; empty string means no failure recorded. + */ +public final class MetaCacheEntryStats { + private final boolean configEnabled; + private final boolean effectiveEnabled; + private final boolean autoRefresh; + private final long ttlSecond; + private final long capacity; + private final long estimatedSize; + private final long requestCount; + private final long hitCount; + private final long missCount; + private final double hitRate; + private final long loadSuccessCount; + private final long loadFailureCount; + private final long totalLoadTimeNanos; + private final double averageLoadPenaltyNanos; + private final long evictionCount; + private final long invalidateCount; + private final long lastLoadSuccessTimeMs; + private final long lastLoadFailureTimeMs; + private final String lastError; + + /** + * Build an immutable stats snapshot. + */ + public MetaCacheEntryStats( + boolean configEnabled, + boolean effectiveEnabled, + boolean autoRefresh, + long ttlSecond, + long capacity, + long estimatedSize, + long requestCount, + long hitCount, + long missCount, + double hitRate, + long loadSuccessCount, + long loadFailureCount, + long totalLoadTimeNanos, + double averageLoadPenaltyNanos, + long evictionCount, + long invalidateCount, + long lastLoadSuccessTimeMs, + long lastLoadFailureTimeMs, + String lastError) { + this.configEnabled = configEnabled; + this.effectiveEnabled = effectiveEnabled; + this.autoRefresh = autoRefresh; + this.ttlSecond = ttlSecond; + this.capacity = capacity; + this.estimatedSize = estimatedSize; + this.requestCount = requestCount; + this.hitCount = hitCount; + this.missCount = missCount; + this.hitRate = hitRate; + this.loadSuccessCount = loadSuccessCount; + this.loadFailureCount = loadFailureCount; + this.totalLoadTimeNanos = totalLoadTimeNanos; + this.averageLoadPenaltyNanos = averageLoadPenaltyNanos; + this.evictionCount = evictionCount; + this.invalidateCount = invalidateCount; + this.lastLoadSuccessTimeMs = lastLoadSuccessTimeMs; + this.lastLoadFailureTimeMs = lastLoadFailureTimeMs; + this.lastError = Objects.requireNonNull(lastError, "lastError"); + } + + public boolean isConfigEnabled() { + return configEnabled; + } + + /** + * Effective cache enable state evaluated by {@link CacheSpec#isCacheEnabled(boolean, long, long)}. + */ + public boolean isEffectiveEnabled() { + return effectiveEnabled; + } + + public boolean isAutoRefresh() { + return autoRefresh; + } + + public long getTtlSecond() { + return ttlSecond; + } + + public long getCapacity() { + return capacity; + } + + public long getEstimatedSize() { + return estimatedSize; + } + + public long getRequestCount() { + return requestCount; + } + + public long getHitCount() { + return hitCount; + } + + public long getMissCount() { + return missCount; + } + + public double getHitRate() { + return hitRate; + } + + public long getLoadSuccessCount() { + return loadSuccessCount; + } + + public long getLoadFailureCount() { + return loadFailureCount; + } + + public long getTotalLoadTimeNanos() { + return totalLoadTimeNanos; + } + + /** + * Average load penalty in nanoseconds. + */ + public double getAverageLoadPenaltyNanos() { + return averageLoadPenaltyNanos; + } + + public long getEvictionCount() { + return evictionCount; + } + + public long getInvalidateCount() { + return invalidateCount; + } + + /** + * Last successful load timestamp in epoch milliseconds, or {@code -1} if absent. + */ + public long getLastLoadSuccessTimeMs() { + return lastLoadSuccessTimeMs; + } + + /** + * Last failed load timestamp in epoch milliseconds, or {@code -1} if absent. + */ + public long getLastLoadFailureTimeMs() { + return lastLoadFailureTimeMs; + } + + /** + * Latest load failure message, or empty string if no failure is recorded. + */ + public String getLastError() { + return lastError; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/paimon/PaimonLatestSnapshotProjectionLoader.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/paimon/PaimonLatestSnapshotProjectionLoader.java new file mode 100644 index 00000000000000..8e2c7a73b33901 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/paimon/PaimonLatestSnapshotProjectionLoader.java @@ -0,0 +1,83 @@ +// 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.datasource.metacache.paimon; + +import org.apache.doris.catalog.Column; +import org.apache.doris.datasource.CacheException; +import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.paimon.PaimonPartitionInfo; +import org.apache.doris.datasource.paimon.PaimonSchemaCacheValue; +import org.apache.doris.datasource.paimon.PaimonSnapshot; +import org.apache.doris.datasource.paimon.PaimonSnapshotCacheValue; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.Snapshot; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.DataTable; +import org.apache.paimon.table.Table; + +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +/** + * Resolves the latest snapshot runtime projection from the base table entry. + */ +public final class PaimonLatestSnapshotProjectionLoader { + @FunctionalInterface + public interface SchemaValueLoader { + PaimonSchemaCacheValue load(NameMapping nameMapping, long schemaId); + } + + private final PaimonPartitionInfoLoader partitionInfoLoader; + private final SchemaValueLoader schemaValueLoader; + + public PaimonLatestSnapshotProjectionLoader(PaimonPartitionInfoLoader partitionInfoLoader, + SchemaValueLoader schemaValueLoader) { + this.partitionInfoLoader = partitionInfoLoader; + this.schemaValueLoader = schemaValueLoader; + } + + public PaimonSnapshotCacheValue load(NameMapping nameMapping, Table paimonTable) { + try { + PaimonSnapshot latestSnapshot = resolveLatestSnapshot(paimonTable); + List partitionColumns = schemaValueLoader.load(nameMapping, latestSnapshot.getSchemaId()) + .getPartitionColumns(); + PaimonPartitionInfo partitionInfo = partitionInfoLoader.load(nameMapping, paimonTable, partitionColumns); + return new PaimonSnapshotCacheValue(partitionInfo, latestSnapshot); + } catch (Exception e) { + throw new CacheException("failed to load paimon snapshot %s.%s.%s: %s", + e, nameMapping.getCtlId(), nameMapping.getLocalDbName(), nameMapping.getLocalTblName(), + e.getMessage()); + } + } + + private PaimonSnapshot resolveLatestSnapshot(Table paimonTable) { + Table snapshotTable = paimonTable; + long latestSnapshotId = PaimonSnapshot.INVALID_SNAPSHOT_ID; + Optional optionalSnapshot = paimonTable.latestSnapshot(); + if (optionalSnapshot.isPresent()) { + latestSnapshotId = optionalSnapshot.get().id(); + snapshotTable = paimonTable.copy( + Collections.singletonMap(CoreOptions.SCAN_SNAPSHOT_ID.key(), String.valueOf(latestSnapshotId))); + } + DataTable dataTable = (DataTable) paimonTable; + long latestSchemaId = dataTable.schemaManager().latest().map(TableSchema::id).orElse(0L); + return new PaimonSnapshot(latestSnapshotId, latestSchemaId, snapshotTable); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/paimon/PaimonPartitionInfoLoader.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/paimon/PaimonPartitionInfoLoader.java new file mode 100644 index 00000000000000..c29a359b9592d1 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/paimon/PaimonPartitionInfoLoader.java @@ -0,0 +1,58 @@ +// 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.datasource.metacache.paimon; + +import org.apache.doris.catalog.Column; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.datasource.CacheException; +import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.paimon.PaimonPartitionInfo; +import org.apache.doris.datasource.paimon.PaimonUtil; + +import org.apache.commons.collections4.CollectionUtils; +import org.apache.paimon.partition.Partition; +import org.apache.paimon.table.Table; + +import java.util.List; + +/** + * Loads partition info for a snapshot projection from the base Paimon table and catalog metadata. + */ +public final class PaimonPartitionInfoLoader { + private final PaimonTableLoader tableLoader; + + public PaimonPartitionInfoLoader(PaimonTableLoader tableLoader) { + this.tableLoader = tableLoader; + } + + public PaimonPartitionInfo load(NameMapping nameMapping, Table paimonTable, List partitionColumns) + throws AnalysisException { + if (CollectionUtils.isEmpty(partitionColumns)) { + return PaimonPartitionInfo.EMPTY; + } + try { + List paimonPartitions = tableLoader.catalog(nameMapping).getPaimonPartitions(nameMapping); + boolean legacyPartitionName = PaimonUtil.isLegacyPartitionName(paimonTable); + return PaimonUtil.generatePartitionInfo(partitionColumns, paimonPartitions, legacyPartitionName); + } catch (Exception e) { + throw new CacheException("failed to load paimon partition info %s.%s.%s: %s", + e, nameMapping.getCtlId(), nameMapping.getLocalDbName(), nameMapping.getLocalTblName(), + e.getMessage()); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/paimon/PaimonTableLoader.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/paimon/PaimonTableLoader.java new file mode 100644 index 00000000000000..0a134cfd7d7d32 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/metacache/paimon/PaimonTableLoader.java @@ -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. + +package org.apache.doris.datasource.metacache.paimon; + +import org.apache.doris.catalog.Env; +import org.apache.doris.datasource.CacheException; +import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.paimon.PaimonExternalCatalog; + +import org.apache.paimon.table.Table; + +import java.io.IOException; + +/** + * Loads the base Paimon table handle used by cache entries and runtime projections. + */ +public final class PaimonTableLoader { + + public Table load(NameMapping nameMapping) { + try { + return catalog(nameMapping).getPaimonTable(nameMapping); + } catch (Exception e) { + throw new CacheException("failed to load paimon table %s.%s.%s: %s", + e, nameMapping.getCtlId(), nameMapping.getLocalDbName(), nameMapping.getLocalTblName(), + e.getMessage()); + } + } + + public PaimonExternalCatalog catalog(NameMapping nameMapping) throws IOException { + return (PaimonExternalCatalog) Env.getCurrentEnv().getCatalogMgr() + .getCatalogOrException(nameMapping.getCtlId(), id -> new IOException("Catalog not found: " + id)); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalCatalog.java index 87f606dfcca451..f093f4d871590d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalCatalog.java @@ -17,6 +17,7 @@ package org.apache.doris.datasource.paimon; +import org.apache.doris.catalog.Env; import org.apache.doris.common.DdlException; import org.apache.doris.datasource.CatalogProperty; import org.apache.doris.datasource.ExternalCatalog; @@ -37,7 +38,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.Objects; // The subclasses of this class are all deprecated, only for meta persistence compatibility. public class PaimonExternalCatalog extends ExternalCatalog { @@ -172,12 +172,9 @@ public void checkProperties() throws DdlException { @Override public void notifyPropertiesUpdated(Map updatedProps) { super.notifyPropertiesUpdated(updatedProps); - String tableCacheEnable = updatedProps.getOrDefault(PAIMON_TABLE_CACHE_ENABLE, null); - String tableCacheTtl = updatedProps.getOrDefault(PAIMON_TABLE_CACHE_TTL_SECOND, null); - String tableCacheCapacity = updatedProps.getOrDefault(PAIMON_TABLE_CACHE_CAPACITY, null); - if (Objects.nonNull(tableCacheEnable) || Objects.nonNull(tableCacheTtl) - || Objects.nonNull(tableCacheCapacity)) { - PaimonUtils.getPaimonMetadataCache(this).init(); + if (updatedProps.keySet().stream() + .anyMatch(key -> CacheSpec.isMetaCacheKeyForEngine(key, PaimonExternalMetaCache.ENGINE))) { + Env.getCurrentEnv().getExtMetaCacheMgr().removeCatalogByEngine(getId(), PaimonExternalMetaCache.ENGINE); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalMetaCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalMetaCache.java new file mode 100644 index 00000000000000..1d08ba1274e256 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalMetaCache.java @@ -0,0 +1,116 @@ +// 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.datasource.paimon; + +import org.apache.doris.datasource.CacheException; +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.doris.datasource.metacache.AbstractExternalMetaCache; +import org.apache.doris.datasource.metacache.MetaCacheEntryDef; +import org.apache.doris.datasource.metacache.MetaCacheEntryInvalidation; +import org.apache.doris.datasource.metacache.paimon.PaimonLatestSnapshotProjectionLoader; +import org.apache.doris.datasource.metacache.paimon.PaimonPartitionInfoLoader; +import org.apache.doris.datasource.metacache.paimon.PaimonTableLoader; + +import org.apache.paimon.table.Table; + +import java.util.Map; +import java.util.concurrent.ExecutorService; + +/** + * Paimon engine implementation of {@link AbstractExternalMetaCache}. + * + *

Registered entries: + *

    + *
  • {@code table}: loaded Paimon table handle per table mapping
  • + *
  • {@code schema}: schema cache keyed by table identity + schema id
  • + *
+ * + *

Latest snapshot metadata is modeled as a runtime projection memoized inside the table cache + * value instead of as an independent cache entry. + * + *

Invalidation behavior: + *

    + *
  • db/table invalidation clears table and schema entries by matching local names
  • + *
  • partition-level invalidation falls back to table-level invalidation
  • + *
+ */ +public class PaimonExternalMetaCache extends AbstractExternalMetaCache { + public static final String ENGINE = "paimon"; + public static final String ENTRY_TABLE = "table"; + public static final String ENTRY_SCHEMA = "schema"; + + private final EntryHandle tableEntry; + private final EntryHandle schemaEntry; + private final PaimonTableLoader tableLoader; + private final PaimonLatestSnapshotProjectionLoader latestSnapshotProjectionLoader; + + public PaimonExternalMetaCache(ExecutorService refreshExecutor) { + super(ENGINE, refreshExecutor); + tableLoader = new PaimonTableLoader(); + latestSnapshotProjectionLoader = new PaimonLatestSnapshotProjectionLoader( + new PaimonPartitionInfoLoader(tableLoader), this::getPaimonSchemaCacheValue); + tableEntry = registerEntry(MetaCacheEntryDef.of(ENTRY_TABLE, NameMapping.class, PaimonTableCacheValue.class, + this::loadTableCacheValue, defaultEntryCacheSpec(), + MetaCacheEntryInvalidation.forNameMapping(nameMapping -> nameMapping))); + schemaEntry = registerEntry(MetaCacheEntryDef.of(ENTRY_SCHEMA, PaimonSchemaCacheKey.class, + SchemaCacheValue.class, this::loadSchemaCacheValue, defaultSchemaCacheSpec(), + MetaCacheEntryInvalidation.forNameMapping(PaimonSchemaCacheKey::getNameMapping))); + } + + public Table getPaimonTable(ExternalTable dorisTable) { + NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); + return tableEntry.get(nameMapping.getCtlId()).get(nameMapping).getPaimonTable(); + } + + public Table getPaimonTable(NameMapping nameMapping) { + return tableEntry.get(nameMapping.getCtlId()).get(nameMapping).getPaimonTable(); + } + + public PaimonSnapshotCacheValue getSnapshotCache(ExternalTable dorisTable) { + NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); + return tableEntry.get(nameMapping.getCtlId()).get(nameMapping).getLatestSnapshotCacheValue(); + } + + public PaimonSchemaCacheValue getPaimonSchemaCacheValue(NameMapping nameMapping, long schemaId) { + SchemaCacheValue schemaCacheValue = schemaEntry.get(nameMapping.getCtlId()) + .get(new PaimonSchemaCacheKey(nameMapping, schemaId)); + return (PaimonSchemaCacheValue) schemaCacheValue; + } + + private PaimonTableCacheValue loadTableCacheValue(NameMapping nameMapping) { + Table paimonTable = tableLoader.load(nameMapping); + return new PaimonTableCacheValue(paimonTable, + () -> latestSnapshotProjectionLoader.load(nameMapping, paimonTable)); + } + + private SchemaCacheValue loadSchemaCacheValue(PaimonSchemaCacheKey key) { + ExternalTable dorisTable = findExternalTable(key.getNameMapping(), ENGINE); + return dorisTable.initSchemaAndUpdateTime(key).orElseThrow(() -> + new CacheException("failed to load paimon schema cache value for: %s.%s.%s, schemaId: %s", + null, key.getNameMapping().getCtlId(), key.getNameMapping().getLocalDbName(), + key.getNameMapping().getLocalTblName(), key.getSchemaId())); + } + + @Override + protected Map catalogPropertyCompatibilityMap() { + return singleCompatibilityMap(ExternalCatalog.SCHEMA_CACHE_TTL_SECOND, ENTRY_SCHEMA); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java index 8e17599c7dd2f0..173379dc56f463 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java @@ -26,8 +26,8 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; import org.apache.doris.datasource.CacheException; -import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey; import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.datasource.SchemaCacheKey; import org.apache.doris.datasource.SchemaCacheValue; import org.apache.doris.datasource.mvcc.MvccSnapshot; import org.apache.doris.datasource.mvcc.MvccTable; @@ -80,6 +80,11 @@ public PaimonExternalTable(long id, String name, String remoteName, PaimonExtern super(id, name, remoteName, catalog, db, TableType.PAIMON_EXTERNAL_TABLE); } + @Override + public String getMetaCacheEngine() { + return PaimonExternalMetaCache.ENGINE; + } + public String getPaimonCatalogType() { return ((PaimonExternalCatalog) catalog).getCatalogType(); } @@ -354,6 +359,11 @@ null, getCatalog().getName(), key.getNameMapping().getLocalDbName(), } } + @Override + public Optional getSchemaCacheValue() { + return Optional.of(getPaimonSchemaCacheValue(MvccUtil.getSnapshotFromContext(this))); + } + private PaimonSchemaCacheValue getPaimonSchemaCacheValue(Optional snapshot) { PaimonSnapshotCacheValue snapshotCacheValue = getOrFetchSnapshotCacheValue(snapshot); return PaimonUtils.getSchemaCacheValue(this, snapshotCacheValue); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCache.java deleted file mode 100644 index 222e5b19ea7197..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCache.java +++ /dev/null @@ -1,229 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.datasource.paimon; - -import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.Env; -import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.CacheFactory; -import org.apache.doris.common.Config; -import org.apache.doris.datasource.CacheException; -import org.apache.doris.datasource.ExternalCatalog; -import org.apache.doris.datasource.ExternalMetaCacheMgr; -import org.apache.doris.datasource.ExternalSchemaCache; -import org.apache.doris.datasource.ExternalTable; -import org.apache.doris.datasource.NameMapping; -import org.apache.doris.datasource.SchemaCacheValue; -import org.apache.doris.datasource.metacache.CacheSpec; - -import com.github.benmanes.caffeine.cache.LoadingCache; -import com.google.common.collect.Maps; -import org.apache.commons.collections4.CollectionUtils; -import org.apache.paimon.CoreOptions; -import org.apache.paimon.Snapshot; -import org.apache.paimon.partition.Partition; -import org.apache.paimon.schema.TableSchema; -import org.apache.paimon.table.DataTable; -import org.apache.paimon.table.Table; -import org.jetbrains.annotations.NotNull; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.OptionalLong; -import java.util.concurrent.ExecutorService; - -public class PaimonMetadataCache { - - private final ExecutorService executor; - private final ExternalCatalog catalog; - private LoadingCache tableCache; - - public PaimonMetadataCache(ExternalCatalog catalog, ExecutorService executor) { - this.executor = executor; - this.catalog = catalog; - init(); - } - - public void init() { - CacheSpec cacheSpec = resolveTableCacheSpec(); - CacheFactory tableCacheFactory = new CacheFactory( - CacheSpec.toExpireAfterAccess(cacheSpec.getTtlSecond()), - OptionalLong.empty(), - cacheSpec.getCapacity(), - true, - null); - this.tableCache = tableCacheFactory.buildCache(key -> loadTableCacheValue(key), executor); - } - - private CacheSpec resolveTableCacheSpec() { - return CacheSpec.fromProperties(catalog.getProperties(), - PaimonExternalCatalog.PAIMON_TABLE_CACHE_ENABLE, true, - PaimonExternalCatalog.PAIMON_TABLE_CACHE_TTL_SECOND, - Config.external_cache_expire_time_seconds_after_access, - PaimonExternalCatalog.PAIMON_TABLE_CACHE_CAPACITY, - Config.max_external_table_cache_num); - } - - @NotNull - private PaimonTableCacheValue loadTableCacheValue(PaimonTableCacheKey key) { - NameMapping nameMapping = key.getNameMapping(); - try { - PaimonExternalCatalog externalCatalog = (PaimonExternalCatalog) Env.getCurrentEnv().getCatalogMgr() - .getCatalogOrException(nameMapping.getCtlId(), - id -> new IOException("Catalog not found: " + id)); - Table table = externalCatalog.getPaimonTable(nameMapping); - return new PaimonTableCacheValue(table); - } catch (Exception e) { - throw new CacheException("failed to load paimon table %s.%s.%s: %s", - e, nameMapping.getCtlId(), nameMapping.getLocalDbName(), - nameMapping.getLocalTblName(), e.getMessage()); - } - } - - @NotNull - private PaimonSnapshotCacheValue loadSnapshot(ExternalTable dorisTable, Table paimonTable) { - NameMapping nameMapping = dorisTable.getOrBuildNameMapping(); - try { - PaimonSnapshot latestSnapshot = loadLatestSnapshot(paimonTable, nameMapping); - List partitionColumns = getPaimonSchemaCacheValue(nameMapping, - latestSnapshot.getSchemaId()).getPartitionColumns(); - boolean legacyPartitionName = PaimonUtil.isLegacyPartitionName(paimonTable); - PaimonPartitionInfo partitionInfo = loadPartitionInfo(nameMapping, partitionColumns, legacyPartitionName); - return new PaimonSnapshotCacheValue(partitionInfo, latestSnapshot); - } catch (Exception e) { - throw new CacheException("failed to load paimon snapshot %s.%s.%s: %s", - e, nameMapping.getCtlId(), nameMapping.getLocalDbName(), - nameMapping.getLocalTblName(), e.getMessage()); - } - } - - public PaimonSchemaCacheValue getPaimonSchemaCacheValue(NameMapping nameMapping, long schemaId) { - ExternalCatalog catalog = (ExternalCatalog) Env.getCurrentEnv().getCatalogMgr() - .getCatalog(nameMapping.getCtlId()); - if (catalog == null) { - throw new CacheException("catalog %s not found when getting paimon schema cache value", - null, nameMapping.getCtlId()); - } - ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); - Optional schemaCacheValue = cache.getSchemaValue( - new PaimonSchemaCacheKey(nameMapping, schemaId)); - if (!schemaCacheValue.isPresent()) { - throw new CacheException("failed to get paimon schema cache value for: %s.%s.%s with schema id: %s", - null, nameMapping.getCtlId(), nameMapping.getLocalDbName(), nameMapping.getLocalTblName(), - schemaId); - } - return (PaimonSchemaCacheValue) schemaCacheValue.get(); - } - - private PaimonPartitionInfo loadPartitionInfo(NameMapping nameMapping, List partitionColumns, - boolean legacyPartitionName) - throws AnalysisException { - if (CollectionUtils.isEmpty(partitionColumns)) { - return PaimonPartitionInfo.EMPTY; - } - PaimonExternalCatalog externalCatalog = (PaimonExternalCatalog) Env.getCurrentEnv().getCatalogMgr() - .getCatalogOrAnalysisException(nameMapping.getCtlId()); - List paimonPartitions = externalCatalog.getPaimonPartitions(nameMapping); - return PaimonUtil.generatePartitionInfo(partitionColumns, paimonPartitions, legacyPartitionName); - } - - private PaimonSnapshot loadLatestSnapshot(Table paimonTable, NameMapping nameMapping) { - Table snapshotTable = paimonTable; - // snapshotId and schemaId - Long latestSnapshotId = PaimonSnapshot.INVALID_SNAPSHOT_ID; - Optional optionalSnapshot = paimonTable.latestSnapshot(); - if (optionalSnapshot.isPresent()) { - latestSnapshotId = optionalSnapshot.get().id(); - snapshotTable = paimonTable.copy( - Collections.singletonMap(CoreOptions.SCAN_SNAPSHOT_ID.key(), latestSnapshotId.toString())); - } - DataTable dataTable = (DataTable) paimonTable; - long latestSchemaId = dataTable.schemaManager().latest().map(TableSchema::id).orElse(0L); - return new PaimonSnapshot(latestSnapshotId, latestSchemaId, snapshotTable); - } - - public Table getPaimonTable(ExternalTable dorisTable) { - PaimonTableCacheKey key = new PaimonTableCacheKey(dorisTable.getOrBuildNameMapping()); - return tableCache.get(key).getPaimonTable(); - } - - public Table getPaimonTable(PaimonTableCacheKey key) { - return tableCache.get(key).getPaimonTable(); - } - - public PaimonSnapshotCacheValue getSnapshotCache(ExternalTable dorisTable) { - PaimonTableCacheKey key = new PaimonTableCacheKey(dorisTable.getOrBuildNameMapping()); - PaimonTableCacheValue tableCacheValue = tableCache.get(key); - return tableCacheValue.getSnapshotCacheValue(() -> loadSnapshot(dorisTable, - tableCacheValue.getPaimonTable())); - } - - public void invalidateCatalogCache(long catalogId) { - tableCache.invalidateAll(); - } - - public void invalidateTableCache(ExternalTable dorisTable) { - PaimonTableCacheKey key = new PaimonTableCacheKey(dorisTable.getOrBuildNameMapping()); - tableCache.invalidate(key); - } - - public void invalidateDbCache(long catalogId, String dbName) { - tableCache.asMap().keySet().stream() - .filter(key -> key.getNameMapping().getLocalDbName().equals(dbName)) - .forEach(tableCache::invalidate); - } - - public Map> getCacheStats() { - Map> res = Maps.newHashMap(); - res.put("paimon_table_cache", ExternalMetaCacheMgr.getCacheStats(tableCache.stats(), - tableCache.estimatedSize())); - return res; - } - - static class PaimonTableCacheKey { - private final NameMapping nameMapping; - - public PaimonTableCacheKey(NameMapping nameMapping) { - this.nameMapping = nameMapping; - } - - public NameMapping getNameMapping() { - return nameMapping; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - PaimonTableCacheKey that = (PaimonTableCacheKey) o; - return nameMapping.equals(that.nameMapping); - } - - @Override - public int hashCode() { - return nameMapping.hashCode(); - } - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheKey.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheKey.java index 2e5456c4432695..4eccb269c2fe56 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheKey.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheKey.java @@ -17,8 +17,8 @@ package org.apache.doris.datasource.paimon; -import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey; import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.SchemaCacheKey; import com.google.common.base.Objects; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSysExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSysExternalTable.java index 3245b9095622bd..8c80511944cf37 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSysExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSysExternalTable.java @@ -19,9 +19,9 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.TableIf; -import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.SchemaCacheKey; import org.apache.doris.datasource.SchemaCacheValue; import org.apache.doris.datasource.systable.SysTable; import org.apache.doris.statistics.AnalysisInfo; @@ -88,6 +88,11 @@ public PaimonSysExternalTable(PaimonExternalTable sourceTable, String sysTableTy this.sysTableType = sysTableType; } + @Override + public String getMetaCacheEngine() { + return PaimonExternalMetaCache.ENGINE; + } + protected synchronized void makeSureInitialized() { super.makeSureInitialized(); if (!objectCreated) { @@ -229,6 +234,11 @@ public Optional initSchema(SchemaCacheKey key) { return Optional.of(new SchemaCacheValue(getFullSchema())); } + @Override + public Optional getSchemaCacheValue() { + return Optional.of(new SchemaCacheValue(getFullSchema())); + } + @Override public Map getSupportedSysTables() { return sourceTable.getSupportedSysTables(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonTableCacheValue.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonTableCacheValue.java index cbbd9076b65a85..7539f28d770bf6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonTableCacheValue.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonTableCacheValue.java @@ -17,45 +17,28 @@ package org.apache.doris.datasource.paimon; +import com.google.common.base.Suppliers; import org.apache.paimon.table.Table; import java.util.function.Supplier; /** - * Cache value for Paimon table metadata. - * Encapsulates the Paimon Table object and provides lazy loading for snapshot cache. + * Cache value for Paimon table metadata and its latest runtime snapshot projection. */ public class PaimonTableCacheValue { private final Table paimonTable; + private final Supplier latestSnapshotCacheValue; - // Lazy-loaded snapshot cache - private volatile boolean snapshotCacheLoaded; - private volatile PaimonSnapshotCacheValue snapshotCacheValue; - - public PaimonTableCacheValue(Table paimonTable) { + public PaimonTableCacheValue(Table paimonTable, Supplier latestSnapshotCacheValue) { this.paimonTable = paimonTable; + this.latestSnapshotCacheValue = Suppliers.memoize(latestSnapshotCacheValue::get); } public Table getPaimonTable() { return paimonTable; } - /** - * Get snapshot cache value with lazy loading. - * Uses double-checked locking to ensure thread-safe initialization. - * - * @param loader Supplier to load snapshot cache value when needed - * @return The cached or newly loaded snapshot cache value - */ - public PaimonSnapshotCacheValue getSnapshotCacheValue(Supplier loader) { - if (!snapshotCacheLoaded) { - synchronized (this) { - if (!snapshotCacheLoaded) { - snapshotCacheValue = loader.get(); - snapshotCacheLoaded = true; - } - } - } - return snapshotCacheValue; + public PaimonSnapshotCacheValue getLatestSnapshotCacheValue() { + return latestSnapshotCacheValue.get(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtils.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtils.java index 30ec4a1185ea82..dc28c083ca10fa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtils.java @@ -18,7 +18,6 @@ package org.apache.doris.datasource.paimon; import org.apache.doris.catalog.Env; -import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.mvcc.MvccSnapshot; @@ -29,11 +28,11 @@ public class PaimonUtils { public static Table getPaimonTable(ExternalTable dorisTable) { - return paimonMetadataCache(dorisTable.getCatalog()).getPaimonTable(dorisTable); + return paimonExternalMetaCache(dorisTable).getPaimonTable(dorisTable); } public static PaimonSnapshotCacheValue getLatestSnapshotCacheValue(ExternalTable dorisTable) { - return paimonMetadataCache(dorisTable.getCatalog()).getSnapshotCache(dorisTable); + return paimonExternalMetaCache(dorisTable).getSnapshotCache(dorisTable); } public static PaimonSnapshotCacheValue getSnapshotCacheValue(Optional snapshot, @@ -50,15 +49,11 @@ public static PaimonSchemaCacheValue getSchemaCacheValue(ExternalTable dorisTabl } public static PaimonSchemaCacheValue getSchemaCacheValue(ExternalTable dorisTable, long schemaId) { - return paimonMetadataCache(dorisTable.getCatalog()) + return paimonExternalMetaCache(dorisTable) .getPaimonSchemaCacheValue(dorisTable.getOrBuildNameMapping(), schemaId); } - public static PaimonMetadataCache getPaimonMetadataCache(ExternalCatalog catalog) { - return paimonMetadataCache(catalog); - } - - private static PaimonMetadataCache paimonMetadataCache(ExternalCatalog catalog) { - return Env.getCurrentEnv().getExtMetaCacheMgr().getPaimonMetadataCache(catalog); + private static PaimonExternalMetaCache paimonExternalMetaCache(ExternalTable table) { + return Env.getCurrentEnv().getExtMetaCacheMgr().paimon(table.getCatalog().getId()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java index d88d94022607a4..b046886a8255b8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java @@ -216,8 +216,7 @@ private void putHistorySchemaInfo(Long schemaId) { } } - TableSchema tableSchema = PaimonUtils.getPaimonMetadataCache(source.getCatalog()) - .getPaimonSchemaCacheValue(targetTable.getOrBuildNameMapping(), schemaId).getTableSchema(); + TableSchema tableSchema = PaimonUtils.getSchemaCacheValue(targetTable, schemaId).getTableSchema(); params.addToHistorySchemaInfo(PaimonUtil.getHistorySchemaInfo(targetTable, tableSchema, source.getCatalog().getEnableMappingVarbinary(), source.getCatalog().getEnableMappingTimestampTz())); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/metastore/AbstractIcebergProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/metastore/AbstractIcebergProperties.java index 8123e6987272d4..ee73f9f6074a22 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/metastore/AbstractIcebergProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/metastore/AbstractIcebergProperties.java @@ -172,13 +172,14 @@ protected void addManifestCacheProperties(Map catalogProps) { // default enable io manifest cache if the meta.cache.manifest is enabled if (!hasIoManifestCacheEnabled) { - CacheSpec manifestCacheSpec = CacheSpec.fromProperties(catalogProps, - IcebergExternalCatalog.ICEBERG_MANIFEST_CACHE_ENABLE, - IcebergExternalCatalog.DEFAULT_ICEBERG_MANIFEST_CACHE_ENABLE, - IcebergExternalCatalog.ICEBERG_MANIFEST_CACHE_TTL_SECOND, - IcebergExternalCatalog.DEFAULT_ICEBERG_MANIFEST_CACHE_TTL_SECOND, - IcebergExternalCatalog.ICEBERG_MANIFEST_CACHE_CAPACITY, - IcebergExternalCatalog.DEFAULT_ICEBERG_MANIFEST_CACHE_CAPACITY); + CacheSpec manifestCacheSpec = CacheSpec.fromProperties(catalogProps, CacheSpec.propertySpecBuilder() + .enable(IcebergExternalCatalog.ICEBERG_MANIFEST_CACHE_ENABLE, + IcebergExternalCatalog.DEFAULT_ICEBERG_MANIFEST_CACHE_ENABLE) + .ttl(IcebergExternalCatalog.ICEBERG_MANIFEST_CACHE_TTL_SECOND, + IcebergExternalCatalog.DEFAULT_ICEBERG_MANIFEST_CACHE_TTL_SECOND) + .capacity(IcebergExternalCatalog.ICEBERG_MANIFEST_CACHE_CAPACITY, + IcebergExternalCatalog.DEFAULT_ICEBERG_MANIFEST_CACHE_CAPACITY) + .build()); if (CacheSpec.isCacheEnabled(manifestCacheSpec.isEnable(), manifestCacheSpec.getTtlSecond(), manifestCacheSpec.getCapacity())) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/test/TestExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/test/TestExternalCatalog.java index 57a483f28c7c58..63e305141c6b88 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/test/TestExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/test/TestExternalCatalog.java @@ -112,4 +112,3 @@ public void gsonPostProcess() throws IOException { initCatalogProvider(); } } - diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/remote/SwitchingFileSystem.java b/fe/fe-core/src/main/java/org/apache/doris/fs/remote/SwitchingFileSystem.java index a6545f2db8020d..6590547694612a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fs/remote/SwitchingFileSystem.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fs/remote/SwitchingFileSystem.java @@ -129,4 +129,3 @@ public FileSystem fileSystem(String location) { return extMetaCacheMgr.getFsCache().getRemoteFileSystem(fileSystemCacheKey); } } - diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SortedPartitionRanges.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SortedPartitionRanges.java index 0c4a1b034b5966..aa2f0219a2f335 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SortedPartitionRanges.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SortedPartitionRanges.java @@ -46,7 +46,7 @@ public SortedPartitionRanges( /** * Build SortedPartitionRanges from a partition map. * This method extracts the common logic for building sorted partition ranges - * from both NereidsSortedPartitionsCacheManager and HiveMetaStoreCache. + * from both NereidsSortedPartitionsCacheManager and HiveExternalMetaCache. * * @param partitionMap map of partition id to partition item * @return SortedPartitionRanges or null if the map is empty diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java index fef4d283165cbe..760a2c0551d5a0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java @@ -23,10 +23,9 @@ import org.apache.doris.common.util.DebugUtil; import org.apache.doris.datasource.ExternalObjectLog; import org.apache.doris.datasource.ExternalTable; -import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.hive.HMSTransaction; -import org.apache.doris.datasource.hive.HiveMetaStoreCache; +import org.apache.doris.datasource.hive.HiveExternalMetaCache; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.qe.ConnectContext; import org.apache.doris.thrift.THivePartitionUpdate; @@ -89,8 +88,8 @@ protected void doAfterCommit() throws DdlException { List modifiedPartNames = Lists.newArrayList(); List newPartNames = Lists.newArrayList(); if (hmsTable.isPartitionedTable() && partitionUpdates != null && !partitionUpdates.isEmpty()) { - HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getMetaStoreCache((HMSExternalCatalog) hmsTable.getCatalog()); + HiveExternalMetaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() + .hive(hmsTable.getCatalog().getId()); cache.refreshAffectedPartitions(hmsTable, partitionUpdates, modifiedPartNames, newPartNames); } else { // Non-partitioned table or no partition updates, do full table refresh diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/HiveTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/HiveTableSink.java index 79da4a51a686e8..a9568e225f0647 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/HiveTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/HiveTableSink.java @@ -26,7 +26,7 @@ import org.apache.doris.common.util.LocationPath; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalTable; -import org.apache.doris.datasource.hive.HiveMetaStoreCache; +import org.apache.doris.datasource.hive.HiveExternalMetaCache; import org.apache.doris.datasource.hive.HiveMetaStoreClientHelper; import org.apache.doris.datasource.hive.HivePartition; import org.apache.doris.datasource.hive.HiveProperties; @@ -208,9 +208,9 @@ private void setPartitionValues(THiveTableSink tSink) throws AnalysisException { List hivePartitions = new ArrayList<>(); if (targetTable.isPartitionedTable()) { // Get partitions from cache instead of HMS client (similar to HiveScanNode) - HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getMetaStoreCache((HMSExternalCatalog) targetTable.getCatalog()); - HiveMetaStoreCache.HivePartitionValues partitionValues = + HiveExternalMetaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() + .hive(targetTable.getCatalog().getId()); + HiveExternalMetaCache.HivePartitionValues partitionValues = targetTable.getHivePartitionValues(MvccUtil.getSnapshotFromContext(targetTable)); List> partitionValuesList = new ArrayList<>(partitionValues.getPartitionValuesMap().values()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/HMSAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/HMSAnalysisTask.java index ea73bda93367f4..3b611f60000cdd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/HMSAnalysisTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/HMSAnalysisTask.java @@ -24,7 +24,7 @@ import org.apache.doris.common.Pair; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.hive.HMSExternalTable; -import org.apache.doris.datasource.hive.HiveMetaStoreCache; +import org.apache.doris.datasource.hive.HiveExternalMetaCache; import org.apache.doris.datasource.hive.HiveUtil; import org.apache.doris.qe.SessionVariable; import org.apache.doris.statistics.util.StatisticsUtil; @@ -111,7 +111,8 @@ private void getPartitionColumnStats() { String value = part[1]; if (colName != null && colName.equals(col.getName())) { // HIVE_DEFAULT_PARTITION hive partition value when the partition name is not specified. - if (value == null || value.isEmpty() || value.equals(HiveMetaStoreCache.HIVE_DEFAULT_PARTITION)) { + if (value == null || value.isEmpty() + || value.equals(HiveExternalMetaCache.HIVE_DEFAULT_PARTITION)) { numNulls += 1; continue; } 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 4dc6df1fc8f52b..e2a0530b02796d 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 @@ -62,12 +62,9 @@ import org.apache.doris.datasource.TablePartitionValues; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalTable; -import org.apache.doris.datasource.hive.HiveMetaStoreCache; -import org.apache.doris.datasource.hudi.source.HudiCachedMetaClientProcessor; -import org.apache.doris.datasource.hudi.source.HudiMetadataCacheMgr; -import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; -import org.apache.doris.datasource.iceberg.IcebergMetadataCache; +import org.apache.doris.datasource.hive.HiveExternalMetaCache; import org.apache.doris.datasource.maxcompute.MaxComputeExternalCatalog; +import org.apache.doris.datasource.metacache.MetaCacheEntryStats; import org.apache.doris.datasource.mvcc.MvccUtil; import org.apache.doris.job.common.JobType; import org.apache.doris.job.extensions.insert.streaming.AbstractStreamingTask; @@ -90,6 +87,7 @@ import org.apache.doris.qe.VariableMgr; import org.apache.doris.resource.workloadgroup.WorkloadGroupMgr; import org.apache.doris.service.ExecuteEnv; +import org.apache.doris.service.FrontendOptions; import org.apache.doris.system.Backend; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.FrontendService; @@ -123,7 +121,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.gson.Gson; -import org.apache.hadoop.conf.Configuration; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.logging.log4j.LogManager; @@ -416,19 +413,16 @@ private static TFetchSchemaTableDataResult hudiMetadataResult(TMetadataTableRequ return errorResult("The specified table is not a hudi table: " + hudiMetadataParams.getTable()); } - HudiCachedMetaClientProcessor hudiMetadataCache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getHudiMetadataCacheMgr().getHudiMetaClientProcessor(catalog); - String hudiBasePathString = ((HMSExternalCatalog) catalog).getClient() - .getTable(dorisTable.getRemoteDbName(), dorisTable.getRemoteName()).getSd().getLocation(); - Configuration conf = ((HMSExternalCatalog) catalog).getConfiguration(); - + HMSExternalTable hudiTable = (HMSExternalTable) dorisTable; List dataBatch = Lists.newArrayList(); TFetchSchemaTableDataResult result = new TFetchSchemaTableDataResult(); switch (hudiQueryType) { case TIMELINE: - HoodieTimeline timeline = hudiMetadataCache.getHoodieTableMetaClient(dorisTable.getOrBuildNameMapping(), - hudiBasePathString, conf).getActiveTimeline(); + HoodieTimeline timeline = Env.getCurrentEnv().getExtMetaCacheMgr() + .hudi(catalog.getId()) + .getHoodieTableMetaClient(hudiTable.getOrBuildNameMapping()) + .getActiveTimeline(); for (HoodieInstant instant : timeline.getInstants()) { TRow trow = new TRow(); trow.addToColumnValue(new TCell().setStringVal(instant.requestedTime())); @@ -1652,21 +1646,59 @@ private static TFetchSchemaTableDataResult metaCacheStatsMetadataResult(TSchemaT List 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()); + String timeZone = VariableMgr.getDefaultSessionVariable().getTimeZone(); + if (params.isSetTimeZone()) { + timeZone = params.getTimeZone(); + } + String feHost = FrontendOptions.getLocalHostAddress(); + UserIdentity currentUserIdentity = params.isSetCurrentUserIdent() + ? UserIdentity.fromThrift(params.getCurrentUserIdent()) + : null; + + List catalogs = Lists.newArrayList(Env.getCurrentEnv().getCatalogMgr().getCopyOfCatalog()); + catalogs.sort((left, right) -> left.getName().compareTo(right.getName())); + for (CatalogIf catalogIf : catalogs) { + if (catalogIf instanceof ExternalCatalog) { + if (currentUserIdentity != null + && !Env.getCurrentEnv().getAccessManager().checkCtlPriv( + currentUserIdentity, catalogIf.getName(), PrivPredicate.SHOW)) { + continue; + } + for (ExternalMetaCacheMgr.CatalogMetaCacheStats cacheStats + : mgr.getCatalogCacheStats(catalogIf.getId())) { + MetaCacheEntryStats entryStats = cacheStats.getEntryStats(); + TRow trow = new TRow(); + trow.addToColumnValue(new TCell().setStringVal(feHost)); // FE_HOST + trow.addToColumnValue(new TCell().setStringVal(catalogIf.getName())); // CATALOG_NAME + trow.addToColumnValue(new TCell().setStringVal(cacheStats.getEngineName())); // ENGINE_NAME + trow.addToColumnValue(new TCell().setStringVal(cacheStats.getEntryName())); // ENTRY_NAME + trow.addToColumnValue(new TCell().setBoolVal(entryStats.isEffectiveEnabled())); // EFFECTIVE_ENABLED + trow.addToColumnValue(new TCell().setBoolVal(entryStats.isConfigEnabled())); // CONFIG_ENABLED + trow.addToColumnValue(new TCell().setBoolVal(entryStats.isAutoRefresh())); // AUTO_REFRESH + trow.addToColumnValue(new TCell().setLongVal(entryStats.getTtlSecond())); // TTL_SECOND + trow.addToColumnValue(new TCell().setLongVal(entryStats.getCapacity())); // CAPACITY + trow.addToColumnValue(new TCell().setLongVal(entryStats.getEstimatedSize())); // ESTIMATED_SIZE + trow.addToColumnValue(new TCell().setLongVal(entryStats.getRequestCount())); // REQUEST_COUNT + trow.addToColumnValue(new TCell().setLongVal(entryStats.getHitCount())); // HIT_COUNT + trow.addToColumnValue(new TCell().setLongVal(entryStats.getMissCount())); // MISS_COUNT + trow.addToColumnValue(new TCell().setDoubleVal(entryStats.getHitRate())); // HIT_RATE + trow.addToColumnValue( + new TCell().setLongVal(entryStats.getLoadSuccessCount())); // LOAD_SUCCESS_COUNT + trow.addToColumnValue( + new TCell().setLongVal(entryStats.getLoadFailureCount())); // LOAD_FAILURE_COUNT + trow.addToColumnValue(new TCell().setLongVal( + TimeUnit.NANOSECONDS.toMillis(entryStats.getTotalLoadTimeNanos()))); // TOTAL_LOAD_TIME_MS + trow.addToColumnValue(new TCell().setDoubleVal( + entryStats.getAverageLoadPenaltyNanos() / TimeUnit.MILLISECONDS.toNanos(1))); + trow.addToColumnValue(new TCell().setLongVal(entryStats.getEvictionCount())); // EVICTION_COUNT + trow.addToColumnValue(new TCell().setLongVal(entryStats.getInvalidateCount())); // INVALIDATE_COUNT + trow.addToColumnValue(new TCell().setStringVal( + formatMetaCacheTime(entryStats.getLastLoadSuccessTimeMs(), timeZone))); + trow.addToColumnValue(new TCell().setStringVal( + formatMetaCacheTime(entryStats.getLastLoadFailureTimeMs(), timeZone))); + trow.addToColumnValue(new TCell().setStringVal(entryStats.getLastError())); // LAST_ERROR + dataBatch.add(trow); } - // 2. hudi cache - HudiMetadataCacheMgr hudiMetadataCacheMgr = mgr.getHudiMetadataCacheMgr(); - fillBatch(dataBatch, hudiMetadataCacheMgr.getCacheStats(catalog), catalog.getName()); - } else if (catalogIf instanceof IcebergExternalCatalog) { - // 3. iceberg cache - IcebergMetadataCache icebergCache = mgr.getIcebergMetadataCache((IcebergExternalCatalog) catalogIf); - fillBatch(dataBatch, icebergCache.getCacheStats(), catalogIf.getName()); } } result.setDataBatch(dataBatch); @@ -1674,6 +1706,13 @@ private static TFetchSchemaTableDataResult metaCacheStatsMetadataResult(TSchemaT return result; } + private static String formatMetaCacheTime(long eventTime, String timeZone) { + if (eventTime < 0) { + return ""; + } + return TimeUtils.longToTimeStringWithTimeZone(eventTime, timeZone); + } + private static void partitionsForInternalCatalog(UserIdentity currentUserIdentity, CatalogIf catalog, DatabaseIf database, List tables, List dataBatch, String timeZone, Long threadId) { for (TableIf table : tables) { @@ -1876,24 +1915,6 @@ private static TFetchSchemaTableDataResult partitionsMetadataResult(TSchemaTable return result; } - private static void fillBatch(List dataBatch, Map> stats, - String catalogName) { - for (Map.Entry> entry : stats.entrySet()) { - String cacheName = entry.getKey(); - Map cacheStats = entry.getValue(); - for (Map.Entry 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); - } - } - } - private static TFetchSchemaTableDataResult partitionValuesMetadataResult(TMetadataTableRequestParams params) { if (!params.isSetPartitionValuesMetadataParams()) { return errorResult("partition values metadata params is not set."); @@ -1943,7 +1964,7 @@ private static List partitionValuesMetadataResultForHmsTable(HMSExternalTa "column " + colNames + " does not match partition columns of table " + tbl.getName()); } - HiveMetaStoreCache.HivePartitionValues hivePartitionValues = tbl.getHivePartitionValues( + HiveExternalMetaCache.HivePartitionValues hivePartitionValues = tbl.getHivePartitionValues( MvccUtil.getSnapshotFromContext(tbl)); Map> valuesMap = hivePartitionValues.getPartitionValuesMap(); List dataBatch = Lists.newArrayList(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/ExternalMetaCacheRouteResolverTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/ExternalMetaCacheRouteResolverTest.java new file mode 100644 index 00000000000000..b9e84c076905d5 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/ExternalMetaCacheRouteResolverTest.java @@ -0,0 +1,376 @@ +// 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.datasource; + +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.datasource.doris.RemoteDorisExternalCatalog; +import org.apache.doris.datasource.hive.HMSExternalCatalog; +import org.apache.doris.datasource.iceberg.IcebergHMSExternalCatalog; +import org.apache.doris.datasource.maxcompute.MaxComputeExternalCatalog; +import org.apache.doris.datasource.metacache.ExternalMetaCache; +import org.apache.doris.datasource.metacache.MetaCacheEntry; +import org.apache.doris.datasource.metacache.MetaCacheEntryStats; +import org.apache.doris.datasource.paimon.PaimonExternalCatalog; + +import mockit.Mock; +import mockit.MockUp; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +public class ExternalMetaCacheRouteResolverTest { + + @Test + public void testEngineAliasCompatibility() { + ExternalMetaCacheMgr metaCacheMgr = new ExternalMetaCacheMgr(true); + Assert.assertEquals("hive", metaCacheMgr.engine("hms").engine()); + Assert.assertEquals("doris", metaCacheMgr.engine("External_Doris").engine()); + Assert.assertEquals("maxcompute", metaCacheMgr.engine("max_compute").engine()); + } + + @Test + public void testRouteByCatalogType() { + ExternalMetaCacheMgr metaCacheMgr = new ExternalMetaCacheMgr(true); + + List hmsEngines = metaCacheMgr.resolveCatalogEngineNamesForTest( + new HMSExternalCatalog(1L, "hms", null, Collections.emptyMap(), ""), 1L); + Assert.assertTrue(hmsEngines.contains("hive")); + Assert.assertTrue(hmsEngines.contains("hudi")); + Assert.assertTrue(hmsEngines.contains("iceberg")); + Assert.assertFalse(hmsEngines.contains("paimon")); + Assert.assertFalse(hmsEngines.contains("doris")); + Assert.assertFalse(hmsEngines.contains("maxcompute")); + Assert.assertFalse(hmsEngines.contains("default")); + + List icebergEngines = metaCacheMgr.resolveCatalogEngineNamesForTest( + new IcebergHMSExternalCatalog(2L, "iceberg", null, Collections.emptyMap(), ""), 2L); + Assert.assertEquals(java.util.Collections.singletonList("iceberg"), icebergEngines); + + List paimonEngines = metaCacheMgr.resolveCatalogEngineNamesForTest( + new PaimonExternalCatalog(3L, "paimon", null, Collections.emptyMap(), ""), 3L); + Assert.assertEquals(java.util.Collections.singletonList("paimon"), paimonEngines); + + List maxComputeEngines = metaCacheMgr.resolveCatalogEngineNamesForTest( + new MaxComputeExternalCatalog(4L, "maxcompute", null, Collections.emptyMap(), ""), 4L); + Assert.assertEquals(java.util.Collections.singletonList("maxcompute"), maxComputeEngines); + + List dorisEngines = metaCacheMgr.resolveCatalogEngineNamesForTest( + new RemoteDorisExternalCatalog(5L, "doris", null, Collections.emptyMap(), ""), 5L); + Assert.assertEquals(java.util.Collections.singletonList("doris"), dorisEngines); + } + + @Test + public void testMissingCatalogOnlyRoutesInitializedEngines() { + ExternalMetaCacheMgr metaCacheMgr = new ExternalMetaCacheMgr(true); + long catalogId = 7L; + + metaCacheMgr.prepareCatalogByEngine(catalogId, "hive", java.util.Collections.emptyMap()); + + List engines = metaCacheMgr.resolveCatalogEngineNamesForTest(null, catalogId); + Assert.assertTrue(engines.contains("hive")); + Assert.assertFalse(engines.contains("iceberg")); + Assert.assertFalse(engines.contains("paimon")); + } + + @Test + public void testPrepareCatalogByEngineSkipsMissingCatalog() throws Exception { + RecordingExternalMetaCache hive = new RecordingExternalMetaCache( + "hive", Collections.singletonList("hms"), catalog -> catalog instanceof HMSExternalCatalog); + ExternalMetaCacheMgr metaCacheMgr = newManagerWithCaches(hive); + long catalogId = 10L; + + mockCurrentCatalog(catalogId, null); + + metaCacheMgr.prepareCatalog(catalogId); + metaCacheMgr.prepareCatalogByEngine(catalogId, "hive"); + + Assert.assertEquals(0, hive.initCatalogCalls); + } + + @Test + public void testGetSchemaCacheValueReturnsEmptyWhenCatalogMissing() throws Exception { + MissingCatalogSchemaExternalMetaCache schemaCache = new MissingCatalogSchemaExternalMetaCache("default"); + ExternalMetaCacheMgr metaCacheMgr = newManagerWithCaches(schemaCache); + long catalogId = 11L; + + mockCurrentCatalog(catalogId, null); + + TestingExternalTable table = new TestingExternalTable(catalogId, "default"); + Assert.assertFalse(metaCacheMgr.getSchemaCacheValue( + table, new SchemaCacheKey(table.getOrBuildNameMapping())).isPresent()); + Assert.assertEquals(1, schemaCache.entryCalls); + } + + @Test + public void testLifecycleRoutingOnlyTouchesSupportedEngine() throws Exception { + RecordingExternalMetaCache hive = new RecordingExternalMetaCache( + "hive", Collections.singletonList("hms"), catalog -> catalog instanceof HMSExternalCatalog); + RecordingExternalMetaCache hudi = new RecordingExternalMetaCache( + "hudi", Collections.emptyList(), catalog -> catalog instanceof HMSExternalCatalog); + RecordingExternalMetaCache iceberg = new RecordingExternalMetaCache( + "iceberg", Collections.emptyList(), catalog -> catalog instanceof HMSExternalCatalog); + RecordingExternalMetaCache paimon = new RecordingExternalMetaCache( + "paimon", Collections.emptyList(), catalog -> catalog instanceof PaimonExternalCatalog); + ExternalMetaCacheMgr metaCacheMgr = newManagerWithCaches(hive, hudi, iceberg, paimon); + long catalogId = 8L; + + HMSExternalCatalog catalog = new HMSExternalCatalog( + catalogId, "hms", null, Collections.singletonMap("k", "v"), ""); + mockCurrentCatalog(catalogId, catalog); + + metaCacheMgr.prepareCatalog(catalogId); + metaCacheMgr.invalidateCatalog(catalogId); + metaCacheMgr.invalidateDb(catalogId, "db1"); + metaCacheMgr.invalidateTable(catalogId, "db1", "tbl1"); + metaCacheMgr.invalidatePartitions(catalogId, "db1", "tbl1", Collections.singletonList("p=1")); + metaCacheMgr.removeCatalog(catalogId); + + Assert.assertEquals(1, hive.initCatalogCalls); + Assert.assertEquals(1, hive.invalidateCatalogEntriesCalls); + Assert.assertEquals(1, hive.invalidateDbCalls); + Assert.assertEquals(1, hive.invalidateTableCalls); + Assert.assertEquals(1, hive.invalidatePartitionsCalls); + Assert.assertEquals(1, hive.invalidateCatalogCalls); + + Assert.assertEquals(1, hudi.initCatalogCalls); + Assert.assertEquals(1, hudi.invalidateCatalogEntriesCalls); + Assert.assertEquals(1, hudi.invalidateDbCalls); + Assert.assertEquals(1, hudi.invalidateTableCalls); + Assert.assertEquals(1, hudi.invalidatePartitionsCalls); + Assert.assertEquals(1, hudi.invalidateCatalogCalls); + + Assert.assertEquals(1, iceberg.initCatalogCalls); + Assert.assertEquals(1, iceberg.invalidateCatalogEntriesCalls); + Assert.assertEquals(1, iceberg.invalidateDbCalls); + Assert.assertEquals(1, iceberg.invalidateTableCalls); + Assert.assertEquals(1, iceberg.invalidatePartitionsCalls); + Assert.assertEquals(1, iceberg.invalidateCatalogCalls); + + Assert.assertEquals(0, paimon.initCatalogCalls); + Assert.assertEquals(0, paimon.invalidateCatalogEntriesCalls); + Assert.assertEquals(0, paimon.invalidateDbCalls); + Assert.assertEquals(0, paimon.invalidateTableCalls); + Assert.assertEquals(0, paimon.invalidatePartitionsCalls); + Assert.assertEquals(0, paimon.invalidateCatalogCalls); + } + + @Test + public void testMissingCatalogLifecycleOnlyTouchesInitializedEngine() throws Exception { + RecordingExternalMetaCache hive = new RecordingExternalMetaCache( + "hive", Collections.singletonList("hms"), catalog -> catalog instanceof HMSExternalCatalog); + RecordingExternalMetaCache paimon = new RecordingExternalMetaCache( + "paimon", Collections.emptyList(), catalog -> catalog instanceof PaimonExternalCatalog); + ExternalMetaCacheMgr metaCacheMgr = newManagerWithCaches(hive, paimon); + long catalogId = 9L; + + hive.initializedCatalogIds.add(catalogId); + mockCurrentCatalog(catalogId, null); + + metaCacheMgr.invalidateCatalog(catalogId); + metaCacheMgr.invalidateDb(catalogId, "db1"); + metaCacheMgr.invalidateTable(catalogId, "db1", "tbl1"); + metaCacheMgr.invalidatePartitions(catalogId, "db1", "tbl1", Collections.singletonList("p=1")); + metaCacheMgr.removeCatalog(catalogId); + + Assert.assertEquals(1, hive.invalidateCatalogEntriesCalls); + Assert.assertEquals(1, hive.invalidateDbCalls); + Assert.assertEquals(1, hive.invalidateTableCalls); + Assert.assertEquals(1, hive.invalidatePartitionsCalls); + Assert.assertEquals(1, hive.invalidateCatalogCalls); + + Assert.assertEquals(0, paimon.invalidateCatalogEntriesCalls); + Assert.assertEquals(0, paimon.invalidateDbCalls); + Assert.assertEquals(0, paimon.invalidateTableCalls); + Assert.assertEquals(0, paimon.invalidatePartitionsCalls); + Assert.assertEquals(0, paimon.invalidateCatalogCalls); + } + + @SuppressWarnings("unchecked") + private ExternalMetaCacheMgr newManagerWithCaches(RecordingExternalMetaCache... caches) throws Exception { + ExternalMetaCacheMgr metaCacheMgr = new ExternalMetaCacheMgr(true); + metaCacheMgr.replaceEngineCachesForTest(java.util.Arrays.asList(caches)); + return metaCacheMgr; + } + + private void mockCurrentCatalog(long catalogId, + CatalogIf> catalog) { + CatalogMgr catalogMgr = new TestingCatalogMgr(catalogId, catalog); + Env env = new TestingEnv(catalogMgr); + new MockUp() { + @Mock + Env getCurrentEnv() { + return env; + } + }; + } + + private static final class TestingCatalogMgr extends CatalogMgr { + private final Map>> catalogs = new HashMap<>(); + + private TestingCatalogMgr(long catalogId, CatalogIf> catalog) { + catalogs.put(catalogId, catalog); + } + + @Override + public CatalogIf> getCatalog(long id) { + return catalogs.get(id); + } + } + + private static final class TestingEnv extends Env { + private final CatalogMgr catalogMgr; + + private TestingEnv(CatalogMgr catalogMgr) { + super(true); + this.catalogMgr = catalogMgr; + } + + @Override + public CatalogMgr getCatalogMgr() { + return catalogMgr; + } + } + + private static final class TestingExternalTable extends ExternalTable { + private final String metaCacheEngine; + + private TestingExternalTable(long catalogId, String metaCacheEngine) { + this.metaCacheEngine = metaCacheEngine; + this.catalog = new HMSExternalCatalog(catalogId, "hms", null, Collections.emptyMap(), ""); + this.dbName = "db1"; + this.name = "tbl1"; + this.remoteName = "remote_tbl1"; + this.nameMapping = new NameMapping(catalogId, "db1", "tbl1", "remote_db1", "remote_tbl1"); + } + + @Override + public String getMetaCacheEngine() { + return metaCacheEngine; + } + } + + private static class RecordingExternalMetaCache implements ExternalMetaCache { + private final String engine; + private final List aliases; + private final Set initializedCatalogIds = ConcurrentHashMap.newKeySet(); + + private int initCatalogCalls; + private int invalidateCatalogCalls; + private int invalidateCatalogEntriesCalls; + private int invalidateDbCalls; + private int invalidateTableCalls; + private int invalidatePartitionsCalls; + + private RecordingExternalMetaCache(String engine, List aliases, + java.util.function.Predicate> ignoredPredicate) { + this.engine = engine; + this.aliases = aliases; + } + + @Override + public String engine() { + return engine; + } + + @Override + public List aliases() { + return aliases; + } + + @Override + public void initCatalog(long catalogId, Map catalogProperties) { + initializedCatalogIds.add(catalogId); + initCatalogCalls++; + } + + @Override + public MetaCacheEntry entry( + long catalogId, String entryName, Class keyType, Class valueType) { + throw new UnsupportedOperationException(); + } + + @Override + public void checkCatalogInitialized(long catalogId) { + if (!isCatalogInitialized(catalogId)) { + throw new IllegalStateException("catalog " + catalogId + " is not initialized"); + } + } + + @Override + public boolean isCatalogInitialized(long catalogId) { + return initializedCatalogIds.contains(catalogId); + } + + @Override + public void invalidateCatalog(long catalogId) { + initializedCatalogIds.remove(catalogId); + invalidateCatalogCalls++; + } + + @Override + public void invalidateCatalogEntries(long catalogId) { + invalidateCatalogEntriesCalls++; + } + + @Override + public void invalidateDb(long catalogId, String dbName) { + invalidateDbCalls++; + } + + @Override + public void invalidateTable(long catalogId, String dbName, String tableName) { + invalidateTableCalls++; + } + + @Override + public void invalidatePartitions(long catalogId, String dbName, String tableName, List partitions) { + invalidatePartitionsCalls++; + } + + @Override + public Map stats(long catalogId) { + return Collections.emptyMap(); + } + + @Override + public void close() { + } + } + + private static final class MissingCatalogSchemaExternalMetaCache extends RecordingExternalMetaCache { + private int entryCalls; + + private MissingCatalogSchemaExternalMetaCache(String engine) { + super(engine, Collections.emptyList(), catalog -> true); + } + + @Override + public MetaCacheEntry entry(long catalogId, String entryName, Class keyType, Class valueType) { + entryCalls++; + throw new IllegalStateException("catalog " + catalogId + " is not initialized"); + } + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/ExternalTableSchemaCacheDelegationTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/ExternalTableSchemaCacheDelegationTest.java new file mode 100644 index 00000000000000..ed375c42fd3da6 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/ExternalTableSchemaCacheDelegationTest.java @@ -0,0 +1,57 @@ +// 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.datasource; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.PrimitiveType; + +import com.google.common.collect.Lists; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; +import java.util.Optional; + +public class ExternalTableSchemaCacheDelegationTest { + + @Test + public void testGetFullSchemaDelegatesToGetSchemaCacheValue() { + List schema = Lists.newArrayList(new Column("c1", PrimitiveType.INT)); + ExternalTable table = new DelegatingExternalTable(Optional.of(new SchemaCacheValue(schema))); + Assert.assertEquals(schema, table.getFullSchema()); + } + + @Test + public void testGetFullSchemaReturnsNullWhenSchemaCacheMissing() { + ExternalTable table = new DelegatingExternalTable(Optional.empty()); + Assert.assertNull(table.getFullSchema()); + } + + private static final class DelegatingExternalTable extends ExternalTable { + private final Optional schemaCacheValue; + + private DelegatingExternalTable(Optional schemaCacheValue) { + this.schemaCacheValue = schemaCacheValue; + } + + @Override + public Optional getSchemaCacheValue() { + return schemaCacheValue; + } + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/PathVisibleTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/PathVisibleTest.java index 6610a5f25218c2..1fd42b1dcb03fd 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/PathVisibleTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/PathVisibleTest.java @@ -17,7 +17,7 @@ package org.apache.doris.datasource; -import org.apache.doris.datasource.hive.HiveMetaStoreCache.FileCacheValue; +import org.apache.doris.datasource.hive.HiveExternalMetaCache.FileCacheValue; import org.apache.hadoop.fs.Path; import org.junit.Assert; diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/doris/DorisExternalMetaCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/doris/DorisExternalMetaCacheTest.java new file mode 100644 index 00000000000000..9257f9d3c20f47 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/doris/DorisExternalMetaCacheTest.java @@ -0,0 +1,62 @@ +// 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.datasource.doris; + +import org.apache.doris.datasource.metacache.MetaCacheEntry; + +import com.google.common.collect.ImmutableMap; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +public class DorisExternalMetaCacheTest { + + @Test + public void testInvalidateBackendCacheUsesSingletonEntryKey() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + DorisExternalMetaCache cache = new DorisExternalMetaCache(executor); + long catalogId = 1L; + cache.initCatalog(catalogId, Collections.emptyMap()); + + MetaCacheEntry> backendsEntry = cache.entry( + catalogId, + DorisExternalMetaCache.ENTRY_BACKENDS, + String.class, + DorisExternalMetaCacheTestSupport.backendMapClass()); + backendsEntry.put("backends", ImmutableMap.of()); + Assert.assertNotNull(backendsEntry.getIfPresent("backends")); + + cache.invalidateBackendCache(catalogId); + + Assert.assertNull(backendsEntry.getIfPresent("backends")); + } finally { + executor.shutdownNow(); + } + } + + private static final class DorisExternalMetaCacheTestSupport { + @SuppressWarnings("unchecked") + private static Class> backendMapClass() { + return (Class>) (Class) ImmutableMap.class; + } + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveAcidTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveAcidTest.java index 2a879a24142fa0..1af66c51164cff 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveAcidTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveAcidTest.java @@ -18,7 +18,7 @@ package org.apache.doris.datasource.hive; import org.apache.doris.datasource.NameMapping; -import org.apache.doris.datasource.hive.HiveMetaStoreCache.FileCacheValue; +import org.apache.doris.datasource.hive.HiveExternalMetaCache.FileCacheValue; import org.apache.doris.datasource.property.storage.LocalProperties; import org.apache.doris.datasource.property.storage.StorageProperties; import org.apache.doris.fs.LocalDfsFileSystem; diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetaStoreCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetaStoreCacheTest.java index f10f7868660088..108a45e03727bf 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetaStoreCacheTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HiveMetaStoreCacheTest.java @@ -20,14 +20,15 @@ import org.apache.doris.common.ThreadPoolManager; import org.apache.doris.common.util.Util; import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.metacache.MetaCacheEntry; -import com.github.benmanes.caffeine.cache.LoadingCache; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.HashMap; import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicLong; public class HiveMetaStoreCacheTest { @@ -38,60 +39,79 @@ public void testInvalidateTableCache() { ThreadPoolExecutor listExecutor = ThreadPoolManager.newDaemonFixedThreadPool( 1, 1, "file", 1, false); - HiveMetaStoreCache hiveMetaStoreCache = new HiveMetaStoreCache( - new HMSExternalCatalog(1L, "catalog", null, new HashMap<>(), null), executor, listExecutor); - - LoadingCache fileCache = hiveMetaStoreCache.getFileCacheRef().get(); - LoadingCache partitionCache = hiveMetaStoreCache.getPartitionCache(); - LoadingCache partitionValuesCache = hiveMetaStoreCache.getPartitionValuesCache(); + HiveExternalMetaCache hiveMetaStoreCache = new HiveExternalMetaCache(executor, listExecutor); + hiveMetaStoreCache.initCatalog(0, new HashMap<>()); + + MetaCacheEntry fileCache = + hiveMetaStoreCache.entry(0, HiveExternalMetaCache.ENTRY_FILE, + HiveExternalMetaCache.FileCacheKey.class, + HiveExternalMetaCache.FileCacheValue.class); + MetaCacheEntry partitionCache = + hiveMetaStoreCache.entry(0, HiveExternalMetaCache.ENTRY_PARTITION, + HiveExternalMetaCache.PartitionCacheKey.class, + HivePartition.class); + MetaCacheEntry + partitionValuesCache = hiveMetaStoreCache.entry(0, HiveExternalMetaCache.ENTRY_PARTITION_VALUES, + HiveExternalMetaCache.PartitionValueCacheKey.class, + HiveExternalMetaCache.HivePartitionValues.class); String dbName = "db"; String tbName = "tb"; String tbName2 = "tb2"; putCache(fileCache, partitionCache, partitionValuesCache, dbName, tbName); - Assertions.assertEquals(2, fileCache.asMap().size()); - Assertions.assertEquals(1, partitionCache.asMap().size()); - Assertions.assertEquals(1, partitionValuesCache.asMap().size()); + Assertions.assertEquals(2, entrySize(fileCache)); + Assertions.assertEquals(1, entrySize(partitionCache)); + Assertions.assertEquals(1, entrySize(partitionValuesCache)); putCache(fileCache, partitionCache, partitionValuesCache, dbName, tbName2); - Assertions.assertEquals(4, fileCache.asMap().size()); - Assertions.assertEquals(2, partitionCache.asMap().size()); - Assertions.assertEquals(2, partitionValuesCache.asMap().size()); + Assertions.assertEquals(4, entrySize(fileCache)); + Assertions.assertEquals(2, entrySize(partitionCache)); + Assertions.assertEquals(2, entrySize(partitionValuesCache)); hiveMetaStoreCache.invalidateTableCache(NameMapping.createForTest(dbName, tbName2)); - Assertions.assertEquals(2, fileCache.asMap().size()); - Assertions.assertEquals(1, partitionCache.asMap().size()); - Assertions.assertEquals(1, partitionValuesCache.asMap().size()); + Assertions.assertEquals(2, entrySize(fileCache)); + Assertions.assertEquals(1, entrySize(partitionCache)); + Assertions.assertEquals(1, entrySize(partitionValuesCache)); hiveMetaStoreCache.invalidateTableCache(NameMapping.createForTest(dbName, tbName)); - Assertions.assertEquals(0, fileCache.asMap().size()); - Assertions.assertEquals(0, partitionCache.asMap().size()); - Assertions.assertEquals(0, partitionValuesCache.asMap().size()); + Assertions.assertEquals(0, entrySize(fileCache)); + Assertions.assertEquals(0, entrySize(partitionCache)); + Assertions.assertEquals(0, entrySize(partitionValuesCache)); } private void putCache( - LoadingCache fileCache, - LoadingCache partitionCache, - LoadingCache partitionValuesCache, + MetaCacheEntry fileCache, + MetaCacheEntry partitionCache, + MetaCacheEntry + partitionValuesCache, String dbName, String tbName) { NameMapping nameMapping = NameMapping.createForTest(dbName, tbName); + long catalogId = nameMapping.getCtlId(); long fileId = Util.genIdByName(dbName, tbName); - HiveMetaStoreCache.FileCacheKey fileCacheKey1 = new HiveMetaStoreCache.FileCacheKey(fileId, tbName, "", new ArrayList<>()); - HiveMetaStoreCache.FileCacheKey fileCacheKey2 = HiveMetaStoreCache.FileCacheKey.createDummyCacheKey(fileId, tbName, ""); - fileCache.put(fileCacheKey1, new HiveMetaStoreCache.FileCacheValue()); - fileCache.put(fileCacheKey2, new HiveMetaStoreCache.FileCacheValue()); - - HiveMetaStoreCache.PartitionCacheKey partitionCacheKey = new HiveMetaStoreCache.PartitionCacheKey( + HiveExternalMetaCache.FileCacheKey fileCacheKey1 = new HiveExternalMetaCache.FileCacheKey( + catalogId, fileId, tbName, "", new ArrayList<>()); + HiveExternalMetaCache.FileCacheKey fileCacheKey2 = HiveExternalMetaCache.FileCacheKey + .createDummyCacheKey(catalogId, fileId, tbName, ""); + fileCache.put(fileCacheKey1, new HiveExternalMetaCache.FileCacheValue()); + fileCache.put(fileCacheKey2, new HiveExternalMetaCache.FileCacheValue()); + + HiveExternalMetaCache.PartitionCacheKey partitionCacheKey = new HiveExternalMetaCache.PartitionCacheKey( nameMapping, new ArrayList<>() ); partitionCache.put(partitionCacheKey, new HivePartition(nameMapping, false, "", "", new ArrayList<>(), new HashMap<>())); - HiveMetaStoreCache.PartitionValueCacheKey partitionValueCacheKey - = new HiveMetaStoreCache.PartitionValueCacheKey(nameMapping, new ArrayList<>()); - partitionValuesCache.put(partitionValueCacheKey, new HiveMetaStoreCache.HivePartitionValues()); + HiveExternalMetaCache.PartitionValueCacheKey partitionValueCacheKey + = new HiveExternalMetaCache.PartitionValueCacheKey(nameMapping, new ArrayList<>()); + partitionValuesCache.put(partitionValueCacheKey, new HiveExternalMetaCache.HivePartitionValues()); + + } + private long entrySize(MetaCacheEntry entry) { + AtomicLong count = new AtomicLong(); + entry.forEach((k, v) -> count.incrementAndGet()); + return count.get(); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/source/HiveScanNodeTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/source/HiveScanNodeTest.java index ba7687157eab19..abb48bcae0b3bc 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/source/HiveScanNodeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/source/HiveScanNodeTest.java @@ -21,7 +21,7 @@ import org.apache.doris.analysis.TupleId; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalTable; -import org.apache.doris.datasource.hive.HiveMetaStoreCache; +import org.apache.doris.datasource.hive.HiveExternalMetaCache; import org.apache.doris.planner.PlanNodeId; import org.apache.doris.planner.ScanContext; import org.apache.doris.qe.SessionVariable; @@ -49,11 +49,11 @@ public void testDetermineTargetFileSplitSizeHonorsMaxFileSplitNum() throws Excep desc.setTable(table); HiveScanNode node = new HiveScanNode(new PlanNodeId(0), desc, false, sv, null, ScanContext.EMPTY); - HiveMetaStoreCache.FileCacheValue fileCacheValue = new HiveMetaStoreCache.FileCacheValue(); - HiveMetaStoreCache.HiveFileStatus status = new HiveMetaStoreCache.HiveFileStatus(); + HiveExternalMetaCache.FileCacheValue fileCacheValue = new HiveExternalMetaCache.FileCacheValue(); + HiveExternalMetaCache.HiveFileStatus status = new HiveExternalMetaCache.HiveFileStatus(); status.setLength(10_000L * MB); fileCacheValue.getFiles().add(status); - List caches = Collections.singletonList(fileCacheValue); + List caches = Collections.singletonList(fileCacheValue); Method method = HiveScanNode.class.getDeclaredMethod( "determineTargetFileSplitSize", List.class, boolean.class); @@ -74,11 +74,11 @@ public void testDetermineTargetFileSplitSizeKeepsInitialSize() throws Exception desc.setTable(table); HiveScanNode node = new HiveScanNode(new PlanNodeId(0), desc, false, sv, null, ScanContext.EMPTY); - HiveMetaStoreCache.FileCacheValue fileCacheValue = new HiveMetaStoreCache.FileCacheValue(); - HiveMetaStoreCache.HiveFileStatus status = new HiveMetaStoreCache.HiveFileStatus(); + HiveExternalMetaCache.FileCacheValue fileCacheValue = new HiveExternalMetaCache.FileCacheValue(); + HiveExternalMetaCache.HiveFileStatus status = new HiveExternalMetaCache.HiveFileStatus(); status.setLength(500L * MB); fileCacheValue.getFiles().add(status); - List caches = Collections.singletonList(fileCacheValue); + List caches = Collections.singletonList(fileCacheValue); Method method = HiveScanNode.class.getDeclaredMethod( "determineTargetFileSplitSize", List.class, boolean.class); diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hudi/HudiExternalMetaCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hudi/HudiExternalMetaCacheTest.java new file mode 100644 index 00000000000000..3932294033d23f --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hudi/HudiExternalMetaCacheTest.java @@ -0,0 +1,188 @@ +// 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.datasource.hudi; + +import org.apache.doris.common.Config; +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.doris.datasource.TablePartitionValues; +import org.apache.doris.datasource.metacache.MetaCacheEntry; +import org.apache.doris.datasource.metacache.MetaCacheEntryStats; + +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +public class HudiExternalMetaCacheTest { + + @Test + public void testEntryAccessAfterExplicitInit() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + HudiExternalMetaCache cache = new HudiExternalMetaCache(executor); + cache.initCatalog(1L, Collections.emptyMap()); + MetaCacheEntry partitionEntry = cache.entry( + 1L, HudiExternalMetaCache.ENTRY_PARTITION, HudiPartitionCacheKey.class, + TablePartitionValues.class); + Assert.assertNotNull(partitionEntry); + cache.checkCatalogInitialized(1L); + } finally { + executor.shutdownNow(); + } + } + + @Test + public void testInvalidateTablePreciseAcrossEntries() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + HudiExternalMetaCache cache = new HudiExternalMetaCache(executor); + long catalogId = 1L; + cache.initCatalog(catalogId, Collections.emptyMap()); + + NameMapping t1 = nameMapping(catalogId, "db1", "tbl1"); + NameMapping t2 = nameMapping(catalogId, "db1", "tbl2"); + + HudiPartitionCacheKey partitionKey1 = partitionKey(t1, 1L, true); + HudiPartitionCacheKey partitionKey2 = partitionKey(t2, 2L, false); + MetaCacheEntry partitionEntry = cache.entry(catalogId, + HudiExternalMetaCache.ENTRY_PARTITION, HudiPartitionCacheKey.class, TablePartitionValues.class); + partitionEntry.put(partitionKey1, new TablePartitionValues()); + partitionEntry.put(partitionKey2, new TablePartitionValues()); + + HudiMetaClientCacheKey metaKey1 = metaClientKey(t1); + HudiMetaClientCacheKey metaKey2 = metaClientKey(t2); + MetaCacheEntry metaClientEntry = cache.entry(catalogId, + HudiExternalMetaCache.ENTRY_META_CLIENT, HudiMetaClientCacheKey.class, HoodieTableMetaClient.class); + metaClientEntry.put(metaKey1, new HoodieTableMetaClient()); + metaClientEntry.put(metaKey2, new HoodieTableMetaClient()); + + HudiSchemaCacheKey schemaKey1 = new HudiSchemaCacheKey(t1, 1L); + HudiSchemaCacheKey schemaKey2 = new HudiSchemaCacheKey(t2, 2L); + MetaCacheEntry schemaEntry = cache.entry(catalogId, + HudiExternalMetaCache.ENTRY_SCHEMA, HudiSchemaCacheKey.class, SchemaCacheValue.class); + schemaEntry.put(schemaKey1, new SchemaCacheValue(Collections.emptyList())); + schemaEntry.put(schemaKey2, new SchemaCacheValue(Collections.emptyList())); + + cache.invalidateTable(catalogId, "db1", "tbl1"); + + Assert.assertNull(partitionEntry.getIfPresent(partitionKey1)); + Assert.assertNotNull(partitionEntry.getIfPresent(partitionKey2)); + Assert.assertNull(metaClientEntry.getIfPresent(metaKey1)); + Assert.assertNotNull(metaClientEntry.getIfPresent(metaKey2)); + Assert.assertNull(schemaEntry.getIfPresent(schemaKey1)); + Assert.assertNotNull(schemaEntry.getIfPresent(schemaKey2)); + } finally { + executor.shutdownNow(); + } + } + + @Test + public void testInvalidatePartitionsFallsBackToTableInvalidation() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + HudiExternalMetaCache cache = new HudiExternalMetaCache(executor); + long catalogId = 1L; + cache.initCatalog(catalogId, Collections.emptyMap()); + + HudiPartitionCacheKey partitionKey1 = partitionKey(nameMapping(catalogId, "db1", "tbl1"), 1L, true); + HudiPartitionCacheKey partitionKey2 = partitionKey(nameMapping(catalogId, "db1", "tbl2"), 2L, false); + MetaCacheEntry partitionEntry = cache.entry(catalogId, + HudiExternalMetaCache.ENTRY_PARTITION, HudiPartitionCacheKey.class, TablePartitionValues.class); + partitionEntry.put(partitionKey1, new TablePartitionValues()); + partitionEntry.put(partitionKey2, new TablePartitionValues()); + + cache.invalidatePartitions(catalogId, "db1", "tbl1", Collections.singletonList("dt=20250101")); + + Assert.assertNull(partitionEntry.getIfPresent(partitionKey1)); + Assert.assertNotNull(partitionEntry.getIfPresent(partitionKey2)); + } finally { + executor.shutdownNow(); + } + } + + @Test + public void testSchemaStatsWhenSchemaCacheDisabled() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + HudiExternalMetaCache cache = new HudiExternalMetaCache(executor); + long catalogId = 1L; + Map properties = com.google.common.collect.Maps.newHashMap(); + properties.put(ExternalCatalog.SCHEMA_CACHE_TTL_SECOND, "0"); + cache.initCatalog(catalogId, properties); + + Map stats = cache.stats(catalogId); + MetaCacheEntryStats schemaStats = stats.get(HudiExternalMetaCache.ENTRY_SCHEMA); + Assert.assertNotNull(schemaStats); + Assert.assertEquals(0L, schemaStats.getTtlSecond()); + Assert.assertTrue(schemaStats.isConfigEnabled()); + Assert.assertFalse(schemaStats.isEffectiveEnabled()); + } finally { + executor.shutdownNow(); + } + } + + @Test + public void testDefaultSpecsFollowConfig() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + long originalExpireAfterAccess = Config.external_cache_expire_time_seconds_after_access; + long originalTableCapacity = Config.max_external_table_cache_num; + long originalSchemaCapacity = Config.max_external_schema_cache_num; + try { + Config.external_cache_expire_time_seconds_after_access = 321L; + Config.max_external_table_cache_num = 7L; + Config.max_external_schema_cache_num = 11L; + + HudiExternalMetaCache cache = new HudiExternalMetaCache(executor); + long catalogId = 1L; + cache.initCatalog(catalogId, Collections.emptyMap()); + + Map stats = cache.stats(catalogId); + MetaCacheEntryStats partitionStats = stats.get(HudiExternalMetaCache.ENTRY_PARTITION); + MetaCacheEntryStats schemaStats = stats.get(HudiExternalMetaCache.ENTRY_SCHEMA); + Assert.assertNotNull(partitionStats); + Assert.assertNotNull(schemaStats); + Assert.assertEquals(321L, partitionStats.getTtlSecond()); + Assert.assertEquals(7L, partitionStats.getCapacity()); + Assert.assertEquals(321L, schemaStats.getTtlSecond()); + Assert.assertEquals(11L, schemaStats.getCapacity()); + } finally { + Config.external_cache_expire_time_seconds_after_access = originalExpireAfterAccess; + Config.max_external_table_cache_num = originalTableCapacity; + Config.max_external_schema_cache_num = originalSchemaCapacity; + executor.shutdownNow(); + } + } + + private NameMapping nameMapping(long catalogId, String dbName, String tableName) { + return new NameMapping(catalogId, dbName, tableName, "remote_" + dbName, "remote_" + tableName); + } + + private HudiPartitionCacheKey partitionKey(NameMapping nameMapping, long timestamp, boolean useHiveSyncPartition) { + return HudiPartitionCacheKey.of(nameMapping, timestamp, useHiveSyncPartition); + } + + private HudiMetaClientCacheKey metaClientKey(NameMapping nameMapping) { + return HudiMetaClientCacheKey.of(nameMapping); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hudi/HudiUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hudi/HudiUtilsTest.java index 070cd2f1859562..71ad096c7ce7c8 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hudi/HudiUtilsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hudi/HudiUtilsTest.java @@ -17,13 +17,20 @@ package org.apache.doris.datasource.hudi; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.CatalogMgr; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalDatabase; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.hive.HiveMetaStoreClientHelper; +import com.google.common.collect.Maps; import mockit.Mock; import mockit.MockUp; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; import org.junit.Assert; @@ -170,9 +177,14 @@ public org.apache.hadoop.hive.metastore.api.Table getRemoteTable() { Files.write(commit1.toPath(), commitContent1.getBytes()); // 3. now, we can get the schema from this table. - HMSExternalCatalog catalog = new HMSExternalCatalog(); + HMSExternalCatalog catalog = new HMSExternalCatalog(10001, "hudi_ut", null, Maps.newHashMap(), ""); + Env env = mockCurrentEnvWithCatalog(catalog); + Assert.assertNotNull(env); + env.getExtMetaCacheMgr().prepareCatalogByEngine(catalog.getId(), HudiExternalMetaCache.ENGINE, + catalog.getProperties()); HMSExternalDatabase db = new HMSExternalDatabase(catalog, 1, "db", "db"); HMSExternalTable hmsExternalTable = new HMSExternalTable(2, "tb", "tb", catalog, db); + mockCatalogLookup(catalog, db, hmsExternalTable); HiveMetaStoreClientHelper.getHudiTableSchema(hmsExternalTable, new boolean[] {false}, "20241219214518880"); // 4. delete the commit file, @@ -194,5 +206,65 @@ public org.apache.hadoop.hive.metastore.api.Table getRemoteTable() { Assert.assertTrue(prop.delete()); Assert.assertTrue(meta.delete()); Files.delete(hudiTable); + env.getExtMetaCacheMgr().invalidateCatalogByEngine(catalog.getId(), HudiExternalMetaCache.ENGINE); + } + + private Env mockCurrentEnvWithCatalog(HMSExternalCatalog catalog) { + CatalogMgr catalogMgr = new TestingCatalogMgr(catalog); + Env env = new TestingEnv(catalogMgr); + new MockUp() { + @Mock + Env getCurrentEnv() { + return env; + } + }; + return env; + } + + private void mockCatalogLookup(HMSExternalCatalog catalog, HMSExternalDatabase db, HMSExternalTable table) { + new MockUp(HMSExternalCatalog.class) { + @Mock + public HMSExternalDatabase getDbNullable(String dbName) { + return "db".equals(dbName) ? db : null; + } + + @Mock + public Configuration getConfiguration() { + return new Configuration(); + } + }; + new MockUp(HMSExternalDatabase.class) { + @Mock + public HMSExternalTable getTableNullable(String tableName) { + return "tb".equals(tableName) ? table : null; + } + }; + } + + private static final class TestingCatalogMgr extends CatalogMgr { + private final CatalogIf> catalog; + + private TestingCatalogMgr(CatalogIf> catalog) { + this.catalog = catalog; + } + + @Override + public CatalogIf> getCatalog(long id) { + return catalog.getId() == id ? catalog : null; + } + } + + private static final class TestingEnv extends Env { + private final CatalogMgr catalogMgr; + + private TestingEnv(CatalogMgr catalogMgr) { + super(true); + this.catalogMgr = catalogMgr; + } + + @Override + public CatalogMgr getCatalogMgr() { + return catalogMgr; + } } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergExternalMetaCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergExternalMetaCacheTest.java new file mode 100644 index 00000000000000..00dde9f4cc0a74 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergExternalMetaCacheTest.java @@ -0,0 +1,373 @@ +// 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.datasource.iceberg; + +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.doris.datasource.iceberg.cache.ManifestCacheValue; +import org.apache.doris.datasource.metacache.MetaCacheEntry; +import org.apache.doris.datasource.metacache.MetaCacheEntryStats; + +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Table; +import org.junit.Assert; +import org.junit.Test; + +import java.lang.reflect.Proxy; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +public class IcebergExternalMetaCacheTest { + + @Test + public void testInvalidateTableKeepsManifestCache() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + IcebergExternalMetaCache cache = new IcebergExternalMetaCache(executor); + long catalogId = 1L; + cache.initCatalog(catalogId, manifestCacheEnabledProperties()); + NameMapping t1 = new NameMapping(catalogId, "db1", "tbl1", "rdb1", "rtbl1"); + NameMapping t2 = new NameMapping(catalogId, "db1", "tbl2", "rdb1", "rtbl2"); + + MetaCacheEntry tableEntry = cache.entry(catalogId, + IcebergExternalMetaCache.ENTRY_TABLE, NameMapping.class, IcebergTableCacheValue.class); + tableEntry.put(t1, new IcebergTableCacheValue(newInterfaceProxy(Table.class), + () -> new IcebergSnapshotCacheValue(IcebergPartitionInfo.empty(), new IcebergSnapshot(1L, 1L)))); + tableEntry.put(t2, new IcebergTableCacheValue(newInterfaceProxy(Table.class), + () -> new IcebergSnapshotCacheValue(IcebergPartitionInfo.empty(), new IcebergSnapshot(2L, 2L)))); + + MetaCacheEntry viewEntry = cache.entry(catalogId, + IcebergExternalMetaCache.ENTRY_VIEW, NameMapping.class, org.apache.iceberg.view.View.class); + viewEntry.put(t1, newInterfaceProxy(org.apache.iceberg.view.View.class)); + viewEntry.put(t2, newInterfaceProxy(org.apache.iceberg.view.View.class)); + + String sharedManifestPath = "/tmp/shared.avro"; + IcebergManifestEntryKey m1 = mockManifestKey(sharedManifestPath); + IcebergManifestEntryKey m2 = mockManifestKey(sharedManifestPath); + MetaCacheEntry manifestEntry = cache.entry(catalogId, + IcebergExternalMetaCache.ENTRY_MANIFEST, IcebergManifestEntryKey.class, ManifestCacheValue.class); + Assert.assertEquals(m1, m2); + manifestEntry.put(m1, ManifestCacheValue.forDataFiles(com.google.common.collect.Lists.newArrayList())); + manifestEntry.put(m2, ManifestCacheValue.forDataFiles(com.google.common.collect.Lists.newArrayList())); + + Assert.assertNotNull(manifestEntry.getIfPresent(m1)); + Assert.assertNotNull(manifestEntry.getIfPresent(m2)); + cache.invalidateTable(catalogId, "db1", "tbl1"); + + Assert.assertNull(tableEntry.getIfPresent(t1)); + Assert.assertNotNull(tableEntry.getIfPresent(t2)); + Assert.assertNull(viewEntry.getIfPresent(t1)); + Assert.assertNotNull(viewEntry.getIfPresent(t2)); + Assert.assertNotNull(manifestEntry.getIfPresent(m1)); + Assert.assertNotNull(manifestEntry.getIfPresent(m2)); + } finally { + executor.shutdownNow(); + } + } + + @Test + public void testInvalidateDbAndStats() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + IcebergExternalMetaCache cache = new IcebergExternalMetaCache(executor); + long catalogId = 1L; + cache.initCatalog(catalogId, manifestCacheEnabledProperties()); + NameMapping db1Table = new NameMapping(catalogId, "db1", "tbl1", "rdb1", "rtbl1"); + NameMapping db2Table = new NameMapping(catalogId, "db2", "tbl1", "rdb2", "rtbl1"); + + MetaCacheEntry tableEntry = cache.entry(catalogId, + IcebergExternalMetaCache.ENTRY_TABLE, NameMapping.class, IcebergTableCacheValue.class); + tableEntry.put(db1Table, new IcebergTableCacheValue(newInterfaceProxy(Table.class), + () -> new IcebergSnapshotCacheValue(IcebergPartitionInfo.empty(), new IcebergSnapshot(1L, 1L)))); + tableEntry.put(db2Table, new IcebergTableCacheValue(newInterfaceProxy(Table.class), + () -> new IcebergSnapshotCacheValue(IcebergPartitionInfo.empty(), new IcebergSnapshot(2L, 2L)))); + + MetaCacheEntry schemaEntry = cache.entry(catalogId, + IcebergExternalMetaCache.ENTRY_SCHEMA, IcebergSchemaCacheKey.class, SchemaCacheValue.class); + IcebergSchemaCacheKey db1Schema = new IcebergSchemaCacheKey(db1Table, 1L); + IcebergSchemaCacheKey db2Schema = new IcebergSchemaCacheKey(db2Table, 2L); + schemaEntry.put(db1Schema, new SchemaCacheValue(Collections.emptyList())); + schemaEntry.put(db2Schema, new SchemaCacheValue(Collections.emptyList())); + MetaCacheEntry manifestEntry = cache.entry(catalogId, + IcebergExternalMetaCache.ENTRY_MANIFEST, IcebergManifestEntryKey.class, ManifestCacheValue.class); + IcebergManifestEntryKey manifestKey = mockManifestKey("/tmp/db-invalidate.avro"); + manifestEntry.put(manifestKey, + ManifestCacheValue.forDataFiles(com.google.common.collect.Lists.newArrayList())); + + cache.invalidateDb(catalogId, "db1"); + + Assert.assertNull(tableEntry.getIfPresent(db1Table)); + Assert.assertNotNull(tableEntry.getIfPresent(db2Table)); + Assert.assertNull(schemaEntry.getIfPresent(db1Schema)); + Assert.assertNotNull(schemaEntry.getIfPresent(db2Schema)); + Assert.assertNotNull(manifestEntry.getIfPresent(manifestKey)); + + Map stats = cache.stats(catalogId); + Assert.assertTrue(stats.containsKey(IcebergExternalMetaCache.ENTRY_TABLE)); + Assert.assertTrue(stats.get(IcebergExternalMetaCache.ENTRY_MANIFEST).isConfigEnabled()); + Assert.assertTrue(stats.get(IcebergExternalMetaCache.ENTRY_MANIFEST).isEffectiveEnabled()); + Assert.assertFalse(stats.get(IcebergExternalMetaCache.ENTRY_MANIFEST).isAutoRefresh()); + Assert.assertEquals(-1L, stats.get(IcebergExternalMetaCache.ENTRY_MANIFEST).getTtlSecond()); + } finally { + executor.shutdownNow(); + } + } + + @Test + public void testSchemaStatsWhenSchemaCacheDisabled() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + IcebergExternalMetaCache cache = new IcebergExternalMetaCache(executor); + long catalogId = 1L; + Map properties = com.google.common.collect.Maps.newHashMap(); + properties.put(ExternalCatalog.SCHEMA_CACHE_TTL_SECOND, "0"); + cache.initCatalog(catalogId, properties); + + Map stats = cache.stats(catalogId); + MetaCacheEntryStats schemaStats = stats.get(IcebergExternalMetaCache.ENTRY_SCHEMA); + Assert.assertNotNull(schemaStats); + Assert.assertEquals(0L, schemaStats.getTtlSecond()); + Assert.assertTrue(schemaStats.isConfigEnabled()); + Assert.assertFalse(schemaStats.isEffectiveEnabled()); + } finally { + executor.shutdownNow(); + } + } + + @Test + public void testManifestStatsDisabledByDefault() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + IcebergExternalMetaCache cache = new IcebergExternalMetaCache(executor); + long catalogId = 1L; + cache.initCatalog(catalogId, Collections.emptyMap()); + + Map stats = cache.stats(catalogId); + MetaCacheEntryStats manifestStats = stats.get(IcebergExternalMetaCache.ENTRY_MANIFEST); + Assert.assertNotNull(manifestStats); + Assert.assertFalse(manifestStats.isConfigEnabled()); + Assert.assertFalse(manifestStats.isEffectiveEnabled()); + Assert.assertFalse(manifestStats.isAutoRefresh()); + Assert.assertEquals(-1L, manifestStats.getTtlSecond()); + Assert.assertEquals(100000L, manifestStats.getCapacity()); + } finally { + executor.shutdownNow(); + } + } + + @Test + public void testManifestEntryRequiresContextualLoader() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + IcebergExternalMetaCache cache = new IcebergExternalMetaCache(executor); + long catalogId = 1L; + cache.initCatalog(catalogId, manifestCacheEnabledProperties()); + MetaCacheEntry manifestEntry = cache.entry(catalogId, + IcebergExternalMetaCache.ENTRY_MANIFEST, IcebergManifestEntryKey.class, ManifestCacheValue.class); + IcebergManifestEntryKey manifestKey = mockManifestKey("/tmp/contextual-only.avro"); + + UnsupportedOperationException exception = Assert.assertThrows(UnsupportedOperationException.class, + () -> manifestEntry.get(manifestKey)); + Assert.assertTrue(exception.getMessage().contains("contextual miss loader")); + + ManifestCacheValue value = manifestEntry.get(manifestKey, + ignored -> ManifestCacheValue.forDataFiles(com.google.common.collect.Lists.newArrayList())); + Assert.assertNotNull(value); + Assert.assertSame(value, manifestEntry.getIfPresent(manifestKey)); + } finally { + executor.shutdownNow(); + } + } + + @Test + public void testManifestEnableUsesDefaultCapacity() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + IcebergExternalMetaCache cache = new IcebergExternalMetaCache(executor); + long catalogId = 1L; + Map properties = com.google.common.collect.Maps.newHashMap(); + properties.put("meta.cache.iceberg.manifest.enable", "true"); + cache.initCatalog(catalogId, properties); + + Map stats = cache.stats(catalogId); + MetaCacheEntryStats manifestStats = stats.get(IcebergExternalMetaCache.ENTRY_MANIFEST); + Assert.assertNotNull(manifestStats); + Assert.assertTrue(manifestStats.isConfigEnabled()); + Assert.assertTrue(manifestStats.isEffectiveEnabled()); + Assert.assertEquals(-1L, manifestStats.getTtlSecond()); + Assert.assertEquals(100000L, manifestStats.getCapacity()); + } finally { + executor.shutdownNow(); + } + } + + private Map manifestCacheEnabledProperties() { + Map properties = com.google.common.collect.Maps.newHashMap(); + properties.put("meta.cache.iceberg.manifest.enable", "true"); + return properties; + } + + private IcebergManifestEntryKey mockManifestKey(String path) { + return IcebergManifestEntryKey.of(new TestingManifestFile(path, ManifestContent.DATA)); + } + + private T newInterfaceProxy(Class type) { + return type.cast(Proxy.newProxyInstance(type.getClassLoader(), new Class[] {type}, (proxy, method, args) -> { + if (method.getDeclaringClass() == Object.class) { + switch (method.getName()) { + case "equals": + return proxy == args[0]; + case "hashCode": + return System.identityHashCode(proxy); + case "toString": + return type.getSimpleName() + "Proxy"; + default: + return null; + } + } + return defaultValue(method.getReturnType()); + })); + } + + private Object defaultValue(Class type) { + if (!type.isPrimitive()) { + return null; + } + if (type == boolean.class) { + return false; + } + if (type == byte.class) { + return (byte) 0; + } + if (type == short.class) { + return (short) 0; + } + if (type == int.class) { + return 0; + } + if (type == long.class) { + return 0L; + } + if (type == float.class) { + return 0F; + } + if (type == double.class) { + return 0D; + } + if (type == char.class) { + return '\0'; + } + throw new IllegalArgumentException("unsupported primitive type: " + type); + } + + private static final class TestingManifestFile implements ManifestFile { + private final String path; + private final ManifestContent content; + + private TestingManifestFile(String path, ManifestContent content) { + this.path = path; + this.content = content; + } + + @Override + public String path() { + return path; + } + + @Override + public ManifestContent content() { + return content; + } + + @Override + public long length() { + return 0; + } + + @Override + public int partitionSpecId() { + return 0; + } + + @Override + public long sequenceNumber() { + return 0; + } + + @Override + public long minSequenceNumber() { + return 0; + } + + @Override + public Long snapshotId() { + return null; + } + + @Override + public Integer addedFilesCount() { + return null; + } + + @Override + public Long addedRowsCount() { + return null; + } + + @Override + public Integer existingFilesCount() { + return null; + } + + @Override + public Long existingRowsCount() { + return null; + } + + @Override + public Integer deletedFilesCount() { + return null; + } + + @Override + public Long deletedRowsCount() { + return null; + } + + @Override + public List partitions() { + return null; + } + + @Override + public ByteBuffer keyMetadata() { + return null; + } + + @Override + public ManifestFile copy() { + return new TestingManifestFile(path, content); + } + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalMetaCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalMetaCacheTest.java new file mode 100644 index 00000000000000..e4e7f6a940626f --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalMetaCacheTest.java @@ -0,0 +1,111 @@ +// 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.datasource.maxcompute; + +import org.apache.doris.catalog.Type; +import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.SchemaCacheKey; +import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.doris.datasource.TablePartitionValues; +import org.apache.doris.datasource.metacache.MetaCacheEntry; +import org.apache.doris.datasource.metacache.MetaCacheEntryStats; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +public class MaxComputeExternalMetaCacheTest { + + @Test + public void testPartitionValuesLoadFromSchemaEntryInsideEngineCache() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + MaxComputeExternalMetaCache cache = new MaxComputeExternalMetaCache(executor); + long catalogId = 1L; + cache.initCatalog(catalogId, Collections.emptyMap()); + + NameMapping table = new NameMapping(catalogId, "db1", "tbl1", "remote_db1", "remote_tbl1"); + MetaCacheEntry schemaEntry = cache.entry( + catalogId, MaxComputeExternalMetaCache.ENTRY_SCHEMA, SchemaCacheKey.class, SchemaCacheValue.class); + schemaEntry.put(new SchemaCacheKey(table), new MaxComputeSchemaCacheValue( + Collections.emptyList(), + null, + null, + Collections.singletonList("pt"), + Collections.singletonList("pt=20250101"), + Collections.emptyList(), + Collections.singletonList(Type.INT), + Collections.emptyMap())); + + TablePartitionValues partitionValues = cache.getPartitionValues(table); + + Assert.assertEquals(1, partitionValues.getPartitionNameToIdMap().size()); + Assert.assertTrue(partitionValues.getPartitionNameToIdMap().containsKey("pt=20250101")); + } finally { + executor.shutdownNow(); + } + } + + @Test + public void testInvalidateTablePrecise() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + MaxComputeExternalMetaCache cache = new MaxComputeExternalMetaCache(executor); + long catalogId = 1L; + cache.initCatalog(catalogId, Collections.emptyMap()); + + NameMapping t1 = new NameMapping(catalogId, "db1", "tbl1", "remote_db1", "remote_tbl1"); + NameMapping t2 = new NameMapping(catalogId, "db1", "tbl2", "remote_db1", "remote_tbl2"); + + MetaCacheEntry partitionEntry = cache.entry( + catalogId, + MaxComputeExternalMetaCache.ENTRY_PARTITION_VALUES, + NameMapping.class, + TablePartitionValues.class); + partitionEntry.put(t1, new TablePartitionValues()); + partitionEntry.put(t2, new TablePartitionValues()); + + cache.invalidateTable(catalogId, "db1", "tbl1"); + + Assert.assertNull(partitionEntry.getIfPresent(t1)); + Assert.assertNotNull(partitionEntry.getIfPresent(t2)); + } finally { + executor.shutdownNow(); + } + } + + @Test + public void testStatsIncludePartitionValuesEntry() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + MaxComputeExternalMetaCache cache = new MaxComputeExternalMetaCache(executor); + long catalogId = 1L; + cache.initCatalog(catalogId, Collections.emptyMap()); + + Map stats = cache.stats(catalogId); + Assert.assertTrue(stats.containsKey(MaxComputeExternalMetaCache.ENTRY_PARTITION_VALUES)); + Assert.assertTrue(stats.containsKey(MaxComputeExternalMetaCache.ENTRY_SCHEMA)); + } finally { + executor.shutdownNow(); + } + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/metacache/AbstractExternalMetaCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/metacache/AbstractExternalMetaCacheTest.java new file mode 100644 index 00000000000000..6407d540d1ef03 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/metacache/AbstractExternalMetaCacheTest.java @@ -0,0 +1,138 @@ +// 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.datasource.metacache; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.SchemaCacheKey; +import org.apache.doris.datasource.SchemaCacheValue; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.junit.Assert; +import org.junit.Test; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +public class AbstractExternalMetaCacheTest { + + @Test + public void testEntryRequiresExplicitInit() { + ExecutorService refreshExecutor = Executors.newSingleThreadExecutor(); + try { + TestExternalMetaCache cache = new TestExternalMetaCache(refreshExecutor); + Assert.assertThrows(IllegalStateException.class, () -> cache.entry( + 1L, "schema", SchemaCacheKey.class, SchemaCacheValue.class)); + + cache.initCatalog(1L, Maps.newHashMap()); + Assert.assertNotNull(cache.entry(1L, "schema", SchemaCacheKey.class, SchemaCacheValue.class)); + } finally { + refreshExecutor.shutdownNow(); + } + } + + @Test + public void testCheckCatalogInitializedRequiresExplicitInit() { + ExecutorService refreshExecutor = Executors.newSingleThreadExecutor(); + try { + TestExternalMetaCache cache = new TestExternalMetaCache(refreshExecutor); + Assert.assertThrows(IllegalStateException.class, () -> cache.checkCatalogInitialized(1L)); + cache.initCatalog(1L, Maps.newHashMap()); + cache.checkCatalogInitialized(1L); + } finally { + refreshExecutor.shutdownNow(); + } + } + + @Test + public void testSchemaEntryValidatesDuplicateColumnsOnLoad() { + ExecutorService refreshExecutor = Executors.newSingleThreadExecutor(); + try { + TestExternalMetaCache cache = new TestExternalMetaCache(refreshExecutor); + cache.initCatalog(1L, Maps.newHashMap()); + + MetaCacheEntry schemaEntry = cache.entry( + 1L, "schema", SchemaCacheKey.class, SchemaCacheValue.class); + + IllegalArgumentException exception = Assert.assertThrows( + IllegalArgumentException.class, + () -> schemaEntry.get(new SchemaCacheKey(NameMapping.createForTest(1L, "db1", "tbl1")))); + Assert.assertTrue(exception.getMessage().contains("Duplicate column name found")); + } finally { + refreshExecutor.shutdownNow(); + } + } + + @Test + public void testEntryFailsFastAfterCatalogRemoved() { + ExecutorService refreshExecutor = Executors.newSingleThreadExecutor(); + try { + TestExternalMetaCache cache = new TestExternalMetaCache(refreshExecutor); + cache.initCatalog(1L, Maps.newHashMap()); + cache.invalidateCatalog(1L); + + IllegalStateException exception = Assert.assertThrows(IllegalStateException.class, + () -> cache.entry(1L, "schema", SchemaCacheKey.class, SchemaCacheValue.class)); + Assert.assertTrue(exception.getMessage().contains("not initialized")); + Assert.assertFalse(cache.isCatalogInitialized(1L)); + } finally { + refreshExecutor.shutdownNow(); + } + } + + @Test + public void testEntryLevelInvalidationUsesRegisteredMatcher() { + ExecutorService refreshExecutor = Executors.newSingleThreadExecutor(); + try { + TestExternalMetaCache cache = new TestExternalMetaCache(refreshExecutor); + cache.initCatalog(1L, Maps.newHashMap()); + + MetaCacheEntry schemaEntry = cache.entry( + 1L, "schema", SchemaCacheKey.class, SchemaCacheValue.class); + SchemaCacheKey matched = new SchemaCacheKey(NameMapping.createForTest(1L, "db1", "tbl1")); + SchemaCacheKey unmatched = new SchemaCacheKey(NameMapping.createForTest(1L, "db2", "tbl2")); + schemaEntry.put(matched, new SchemaCacheValue(Lists.newArrayList(new Column("id", PrimitiveType.INT)))); + schemaEntry.put(unmatched, new SchemaCacheValue(Lists.newArrayList(new Column("id", PrimitiveType.INT)))); + + cache.invalidateTable(1L, "db1", "tbl1"); + + Assert.assertNull(schemaEntry.getIfPresent(matched)); + Assert.assertNotNull(schemaEntry.getIfPresent(unmatched)); + Assert.assertTrue(cache.isCatalogInitialized(1L)); + } finally { + refreshExecutor.shutdownNow(); + } + } + + private static final class TestExternalMetaCache extends AbstractExternalMetaCache { + private TestExternalMetaCache(ExecutorService refreshExecutor) { + super("test_engine", refreshExecutor); + registerEntry(MetaCacheEntryDef.of( + "schema", + SchemaCacheKey.class, + SchemaCacheValue.class, + key -> new SchemaCacheValue(Lists.newArrayList( + new Column("id", PrimitiveType.INT), + new Column("ID", PrimitiveType.INT))), + CacheSpec.of(true, CacheSpec.CACHE_NO_TTL, 10L), + MetaCacheEntryInvalidation.forNameMapping(SchemaCacheKey::getNameMapping))); + } + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/metacache/CacheSpecTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/metacache/CacheSpecTest.java new file mode 100644 index 00000000000000..05acbb539a26d6 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/metacache/CacheSpecTest.java @@ -0,0 +1,169 @@ +// 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.datasource.metacache; + +import org.apache.doris.common.DdlException; + +import com.google.common.collect.Maps; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Map; +import java.util.OptionalLong; + +public class CacheSpecTest { + + @Test + public void testFromPropertiesWithExplicitKeys() { + Map properties = Maps.newHashMap(); + properties.put("k.enable", "false"); + properties.put("k.ttl", "123"); + properties.put("k.capacity", "456"); + + CacheSpec spec = CacheSpec.fromProperties( + properties, + "k.enable", true, + "k.ttl", CacheSpec.CACHE_NO_TTL, + "k.capacity", 100); + + Assert.assertFalse(spec.isEnable()); + Assert.assertEquals(123, spec.getTtlSecond()); + Assert.assertEquals(456, spec.getCapacity()); + } + + @Test + public void testFromPropertiesWithPropertySpecBuilder() { + Map properties = Maps.newHashMap(); + properties.put("k.enable", "false"); + properties.put("k.ttl", "123"); + properties.put("k.capacity", "456"); + + CacheSpec spec = CacheSpec.fromProperties(properties, CacheSpec.propertySpecBuilder() + .enable("k.enable", true) + .ttl("k.ttl", CacheSpec.CACHE_NO_TTL) + .capacity("k.capacity", 100) + .build()); + + Assert.assertFalse(spec.isEnable()); + Assert.assertEquals(123, spec.getTtlSecond()); + Assert.assertEquals(456, spec.getCapacity()); + } + + @Test + public void testFromPropertiesWithEngineEntryKeys() { + Map properties = Maps.newHashMap(); + properties.put("meta.cache.hive.schema.ttl-second", "0"); + + CacheSpec defaultSpec = CacheSpec.fromProperties( + Maps.newHashMap(), + "enable", true, + "ttl", 60, + "capacity", 100); + + CacheSpec spec = CacheSpec.fromProperties(properties, "hive", "schema", defaultSpec); + Assert.assertTrue(spec.isEnable()); + Assert.assertEquals(0, spec.getTtlSecond()); + Assert.assertEquals(100, spec.getCapacity()); + } + + @Test + public void testApplyCompatibilityMap() { + Map properties = Maps.newHashMap(); + properties.put("legacy.ttl", "10"); + properties.put("new.ttl", "20"); + properties.put("legacy.capacity", "30"); + + Map compatibilityMap = Maps.newHashMap(); + compatibilityMap.put("legacy.ttl", "new.ttl"); + compatibilityMap.put("legacy.capacity", "new.capacity"); + + Map mapped = CacheSpec.applyCompatibilityMap(properties, compatibilityMap); + + // New key keeps precedence if already present. + Assert.assertEquals("20", mapped.get("new.ttl")); + // Missing new key is copied from legacy key. + Assert.assertEquals("30", mapped.get("new.capacity")); + // Original map is not modified. + Assert.assertFalse(properties.containsKey("new.capacity")); + } + + @Test + public void testOfSemantics() { + CacheSpec enabled = CacheSpec.of(true, 60, 100); + Assert.assertTrue(enabled.isEnable()); + Assert.assertEquals(60, enabled.getTtlSecond()); + Assert.assertEquals(100, enabled.getCapacity()); + + CacheSpec zeroTtl = CacheSpec.of(true, 0, 100); + Assert.assertTrue(zeroTtl.isEnable()); + Assert.assertEquals(0, zeroTtl.getTtlSecond()); + Assert.assertEquals(100, zeroTtl.getCapacity()); + + CacheSpec disabled = CacheSpec.of(false, 60, 100); + Assert.assertFalse(disabled.isEnable()); + Assert.assertEquals(60, disabled.getTtlSecond()); + Assert.assertEquals(100, disabled.getCapacity()); + } + + @Test + public void testPropertyValidationHelpers() throws Exception { + CacheSpec.checkBooleanProperty("true", "k.enable"); + CacheSpec.checkBooleanProperty("false", "k.enable"); + + try { + CacheSpec.checkBooleanProperty("on", "k.enable"); + Assert.fail("expected DdlException"); + } catch (DdlException e) { + Assert.assertTrue(e.getMessage().contains("k.enable")); + } + + CacheSpec.checkLongProperty("10", 0, "k.ttl"); + try { + CacheSpec.checkLongProperty("-1", 0, "k.ttl"); + Assert.fail("expected DdlException"); + } catch (DdlException e) { + Assert.assertTrue(e.getMessage().contains("k.ttl")); + } + } + + @Test + public void testIsCacheEnabled() { + Assert.assertTrue(CacheSpec.isCacheEnabled(true, CacheSpec.CACHE_NO_TTL, 1)); + Assert.assertFalse(CacheSpec.isCacheEnabled(false, CacheSpec.CACHE_NO_TTL, 1)); + Assert.assertFalse(CacheSpec.isCacheEnabled(true, 0, 1)); + Assert.assertFalse(CacheSpec.isCacheEnabled(true, CacheSpec.CACHE_NO_TTL, 0)); + } + + @Test + public void testToExpireAfterAccess() { + OptionalLong noTtl = CacheSpec.toExpireAfterAccess(CacheSpec.CACHE_NO_TTL); + Assert.assertFalse(noTtl.isPresent()); + + OptionalLong disabled = CacheSpec.toExpireAfterAccess(0); + Assert.assertTrue(disabled.isPresent()); + Assert.assertEquals(0, disabled.getAsLong()); + + OptionalLong positive = CacheSpec.toExpireAfterAccess(15); + Assert.assertTrue(positive.isPresent()); + Assert.assertEquals(15, positive.getAsLong()); + + OptionalLong negativeOther = CacheSpec.toExpireAfterAccess(-2); + Assert.assertTrue(negativeOther.isPresent()); + Assert.assertEquals(0, negativeOther.getAsLong()); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/metacache/MetaCacheEntryTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/metacache/MetaCacheEntryTest.java new file mode 100644 index 00000000000000..39ab13fcb7b5dc --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/metacache/MetaCacheEntryTest.java @@ -0,0 +1,199 @@ +// 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.datasource.metacache; + +import com.github.benmanes.caffeine.cache.LoadingCache; +import com.google.common.collect.Maps; +import org.junit.Assert; +import org.junit.Test; + +import java.lang.reflect.Field; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; + +public class MetaCacheEntryTest { + + @Test + public void testRefreshUsesConfiguredLoader() throws Exception { + ExecutorService refreshExecutor = Executors.newSingleThreadExecutor(); + try { + Map properties = Maps.newHashMap(); + CacheSpec cacheSpec = CacheSpec.fromProperties( + properties, + "k.enable", true, + "k.ttl", CacheSpec.CACHE_NO_TTL, + "k.capacity", 10L); + AtomicInteger loadCounter = new AtomicInteger(); + MetaCacheEntry entry = new MetaCacheEntry<>( + "test", + key -> loadCounter.incrementAndGet(), + cacheSpec, + refreshExecutor); + + Assert.assertEquals(Integer.valueOf(1), entry.get("k")); + + LoadingCache loadingCache = extractLoadingCache(entry); + loadingCache.refresh("k"); + + long deadlineMs = System.currentTimeMillis() + 3000L; + while (loadCounter.get() < 2 && System.currentTimeMillis() < deadlineMs) { + Thread.sleep(20L); + } + Assert.assertTrue("refresh should trigger loader invocation", loadCounter.get() >= 2); + } finally { + refreshExecutor.shutdownNow(); + } + } + + @Test + public void testGetWithMissLoaderAndDisableAutoRefresh() throws Exception { + ExecutorService refreshExecutor = Executors.newSingleThreadExecutor(); + try { + Map properties = Maps.newHashMap(); + CacheSpec cacheSpec = CacheSpec.fromProperties( + properties, + "k.enable", true, + "k.ttl", CacheSpec.CACHE_NO_TTL, + "k.capacity", 10L); + AtomicInteger defaultLoaderCounter = new AtomicInteger(); + MetaCacheEntry entry = new MetaCacheEntry<>( + "test", + key -> defaultLoaderCounter.incrementAndGet(), + cacheSpec, + refreshExecutor, + false); + + AtomicInteger missLoaderCounter = new AtomicInteger(); + Assert.assertEquals(Integer.valueOf(100), entry.get("k", key -> 100 + missLoaderCounter.getAndIncrement())); + Assert.assertEquals(Integer.valueOf(100), entry.get("k")); + Assert.assertEquals(1, missLoaderCounter.get()); + Assert.assertEquals(0, defaultLoaderCounter.get()); + + LoadingCache loadingCache = extractLoadingCache(entry); + Assert.assertFalse(loadingCache.policy().refreshAfterWrite().isPresent()); + } finally { + refreshExecutor.shutdownNow(); + } + } + + @Test + public void testStatsSnapshotTracksLoadAndLastError() { + ExecutorService refreshExecutor = Executors.newSingleThreadExecutor(); + try { + Map properties = Maps.newHashMap(); + CacheSpec cacheSpec = CacheSpec.fromProperties( + properties, + "k.enable", true, + "k.ttl", CacheSpec.CACHE_NO_TTL, + "k.capacity", 10L); + + MetaCacheEntry entry = new MetaCacheEntry<>( + "test", + key -> { + if ("fail".equals(key)) { + throw new IllegalStateException("mock failure"); + } + return 1; + }, + cacheSpec, + refreshExecutor, + false); + + Assert.assertEquals(Integer.valueOf(1), entry.get("ok")); + Assert.assertEquals(Integer.valueOf(1), entry.get("ok")); + Assert.assertThrows(IllegalStateException.class, () -> entry.get("fail")); + + MetaCacheEntryStats failedStats = entry.stats(); + Assert.assertEquals(3L, failedStats.getRequestCount()); + Assert.assertEquals(1L, failedStats.getHitCount()); + Assert.assertEquals(2L, failedStats.getMissCount()); + Assert.assertEquals(1L, failedStats.getLoadSuccessCount()); + Assert.assertEquals(1L, failedStats.getLoadFailureCount()); + Assert.assertTrue(failedStats.getLastLoadSuccessTimeMs() > 0); + Assert.assertTrue(failedStats.getLastLoadFailureTimeMs() > 0); + Assert.assertTrue(failedStats.getLastError().contains("mock failure")); + + Assert.assertEquals(Integer.valueOf(101), entry.get("miss-loader", key -> 101)); + MetaCacheEntryStats recoveredStats = entry.stats(); + Assert.assertTrue(recoveredStats.getLastError().contains("mock failure")); + Assert.assertTrue(recoveredStats.getLoadSuccessCount() >= 2L); + } finally { + refreshExecutor.shutdownNow(); + } + } + + @Test + public void testStatsSnapshotContainsEffectiveEnabled() { + ExecutorService refreshExecutor = Executors.newSingleThreadExecutor(); + try { + Map properties = Maps.newHashMap(); + CacheSpec cacheSpec = CacheSpec.fromProperties( + properties, + "k.enable", true, + "k.ttl", 0L, + "k.capacity", 10L); + MetaCacheEntry entry = new MetaCacheEntry<>( + "test", + key -> 1, + cacheSpec, + refreshExecutor, + false); + + MetaCacheEntryStats stats = entry.stats(); + Assert.assertTrue(stats.isConfigEnabled()); + Assert.assertFalse(stats.isEffectiveEnabled()); + Assert.assertEquals(10L, stats.getCapacity()); + Assert.assertEquals(0L, stats.getEstimatedSize()); + } finally { + refreshExecutor.shutdownNow(); + } + } + + @Test + public void testContextualOnlyEntryRejectsDefaultGet() { + ExecutorService refreshExecutor = Executors.newSingleThreadExecutor(); + try { + CacheSpec cacheSpec = CacheSpec.of(true, CacheSpec.CACHE_NO_TTL, 10L); + MetaCacheEntry entry = new MetaCacheEntry<>( + "contextual", + null, + cacheSpec, + refreshExecutor, + false, + true); + + UnsupportedOperationException exception = Assert.assertThrows( + UnsupportedOperationException.class, () -> entry.get("k")); + Assert.assertTrue(exception.getMessage().contains("contextual miss loader")); + Assert.assertEquals(Integer.valueOf(7), entry.get("k", key -> 7)); + } finally { + refreshExecutor.shutdownNow(); + } + } + + @SuppressWarnings("unchecked") + private LoadingCache extractLoadingCache(MetaCacheEntry entry) throws Exception { + Field dataField = MetaCacheEntry.class.getDeclaredField("data"); + dataField.setAccessible(true); + Object raw = dataField.get(entry); + Assert.assertTrue(raw instanceof LoadingCache); + return (LoadingCache) raw; + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/paimon/PaimonExternalMetaCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/paimon/PaimonExternalMetaCacheTest.java new file mode 100644 index 00000000000000..dc33354f6f93fd --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/paimon/PaimonExternalMetaCacheTest.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.datasource.paimon; + +import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.NameMapping; +import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.doris.datasource.metacache.MetaCacheEntryStats; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +public class PaimonExternalMetaCacheTest { + + @Test + public void testInvalidateTablePrecise() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + PaimonExternalMetaCache cache = new PaimonExternalMetaCache(executor); + long catalogId = 1L; + cache.initCatalog(catalogId, Collections.emptyMap()); + NameMapping t1 = new NameMapping(catalogId, "db1", "tbl1", "rdb1", "rtbl1"); + NameMapping t2 = new NameMapping(catalogId, "db1", "tbl2", "rdb1", "rtbl2"); + + org.apache.doris.datasource.metacache.MetaCacheEntry tableEntry = + cache.entry(catalogId, PaimonExternalMetaCache.ENTRY_TABLE, + NameMapping.class, PaimonTableCacheValue.class); + tableEntry.put(t1, new PaimonTableCacheValue(null, + () -> new PaimonSnapshotCacheValue(PaimonPartitionInfo.EMPTY, new PaimonSnapshot(1L, 1L, null)))); + tableEntry.put(t2, new PaimonTableCacheValue(null, + () -> new PaimonSnapshotCacheValue(PaimonPartitionInfo.EMPTY, new PaimonSnapshot(2L, 2L, null)))); + + cache.invalidateTable(catalogId, "db1", "tbl1"); + + Assert.assertNull(tableEntry.getIfPresent(t1)); + Assert.assertNotNull(tableEntry.getIfPresent(t2)); + } finally { + executor.shutdownNow(); + } + } + + @Test + public void testInvalidateDbAndStats() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + PaimonExternalMetaCache cache = new PaimonExternalMetaCache(executor); + long catalogId = 1L; + cache.initCatalog(catalogId, Collections.emptyMap()); + NameMapping db1Table = new NameMapping(catalogId, "db1", "tbl1", "rdb1", "rtbl1"); + NameMapping db2Table = new NameMapping(catalogId, "db2", "tbl1", "rdb2", "rtbl1"); + + org.apache.doris.datasource.metacache.MetaCacheEntry tableEntry = + cache.entry(catalogId, PaimonExternalMetaCache.ENTRY_TABLE, + NameMapping.class, PaimonTableCacheValue.class); + tableEntry.put(db1Table, new PaimonTableCacheValue(null, + () -> new PaimonSnapshotCacheValue(PaimonPartitionInfo.EMPTY, new PaimonSnapshot(1L, 1L, null)))); + tableEntry.put(db2Table, new PaimonTableCacheValue(null, + () -> new PaimonSnapshotCacheValue(PaimonPartitionInfo.EMPTY, new PaimonSnapshot(2L, 2L, null)))); + + org.apache.doris.datasource.metacache.MetaCacheEntry schemaEntry = + cache.entry(catalogId, PaimonExternalMetaCache.ENTRY_SCHEMA, + PaimonSchemaCacheKey.class, SchemaCacheValue.class); + PaimonSchemaCacheKey db1Schema = new PaimonSchemaCacheKey(db1Table, 1L); + PaimonSchemaCacheKey db2Schema = new PaimonSchemaCacheKey(db2Table, 2L); + schemaEntry.put(db1Schema, new SchemaCacheValue(Collections.emptyList())); + schemaEntry.put(db2Schema, new SchemaCacheValue(Collections.emptyList())); + + cache.invalidateDb(catalogId, "db1"); + + Assert.assertNull(tableEntry.getIfPresent(db1Table)); + Assert.assertNotNull(tableEntry.getIfPresent(db2Table)); + Assert.assertNull(schemaEntry.getIfPresent(db1Schema)); + Assert.assertNotNull(schemaEntry.getIfPresent(db2Schema)); + + Map stats = cache.stats(catalogId); + Assert.assertTrue(stats.containsKey(PaimonExternalMetaCache.ENTRY_TABLE)); + Assert.assertTrue(stats.containsKey(PaimonExternalMetaCache.ENTRY_SCHEMA)); + } finally { + executor.shutdownNow(); + } + } + + @Test + public void testSchemaStatsWhenSchemaCacheDisabled() { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + PaimonExternalMetaCache cache = new PaimonExternalMetaCache(executor); + long catalogId = 1L; + Map properties = com.google.common.collect.Maps.newHashMap(); + properties.put(ExternalCatalog.SCHEMA_CACHE_TTL_SECOND, "0"); + cache.initCatalog(catalogId, properties); + + Map stats = cache.stats(catalogId); + MetaCacheEntryStats schemaStats = stats.get(PaimonExternalMetaCache.ENTRY_SCHEMA); + Assert.assertNotNull(schemaStats); + Assert.assertEquals(0L, schemaStats.getTtlSecond()); + Assert.assertTrue(schemaStats.isConfigEnabled()); + Assert.assertFalse(schemaStats.isEffectiveEnabled()); + } finally { + executor.shutdownNow(); + } + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/external/hms/HmsCatalogTest.java b/fe/fe-core/src/test/java/org/apache/doris/external/hms/HmsCatalogTest.java index 43d5669e6ad74a..7fd9f0120cbe83 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/external/hms/HmsCatalogTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/external/hms/HmsCatalogTest.java @@ -25,12 +25,13 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.common.jmockit.Deencapsulation; import org.apache.doris.datasource.CatalogMgr; -import org.apache.doris.datasource.ExternalSchemaCache; import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.datasource.SchemaCacheKey; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalDatabase; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.hive.HMSExternalTable.DLAType; +import org.apache.doris.datasource.hive.HMSSchemaCacheValue; import org.apache.doris.datasource.hive.HiveDlaTable; import org.apache.doris.nereids.datasets.tpch.AnalyzeCheckTestBase; import org.apache.doris.nereids.parser.NereidsParser; @@ -47,6 +48,7 @@ import org.junit.jupiter.api.Test; import java.util.List; +import java.util.Optional; public class HmsCatalogTest extends AnalyzeCheckTestBase { private static final String HMS_CATALOG = "hms_ctl"; @@ -98,9 +100,11 @@ protected void runBeforeAll() throws Exception { private void createDbAndTableForHmsCatalog(HMSExternalCatalog hmsCatalog) { Deencapsulation.setField(hmsCatalog, "initialized", true); Deencapsulation.setField(hmsCatalog, "objectCreated", true); + Env.getCurrentEnv().getExtMetaCacheMgr().prepareCatalog(hmsCatalog.getId()); List schema = Lists.newArrayList(); schema.add(new Column("k1", PrimitiveType.INT)); + HMSSchemaCacheValue schemaCacheValue = new HMSSchemaCacheValue(schema, Lists.newArrayList()); HMSExternalDatabase db = new HMSExternalDatabase(hmsCatalog, 10000, "hms_db", "hms_db"); Deencapsulation.setField(db, "initialized", true); @@ -143,9 +147,17 @@ private void createDbAndTableForHmsCatalog(HMSExternalCatalog hmsCatalog) { minTimes = 0; result = TableIf.TableType.HMS_EXTERNAL_TABLE; - // mock initSchemaAndUpdateTime and do nothing - tbl.initSchemaAndUpdateTime(new ExternalSchemaCache.SchemaCacheKey(tbl.getOrBuildNameMapping())); + tbl.getCatalog(); minTimes = 0; + result = hmsCatalog; + + tbl.getSchemaCacheValue(); + minTimes = 0; + result = Optional.of(schemaCacheValue); + + tbl.initSchemaAndUpdateTime((SchemaCacheKey) any); + minTimes = 0; + result = Optional.of(schemaCacheValue); tbl.getDatabase(); minTimes = 0; diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/LoadManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/LoadManagerTest.java index c88878e032db46..94ff7560655cae 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/LoadManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/LoadManagerTest.java @@ -47,14 +47,19 @@ public class LoadManagerTest { private LoadManager loadManager; private final String fieldName = "idToLoadJob"; private UserIdentity userInfo = UserIdentity.createAnalyzedUserIdentWithIp("root", "localhost"); + private int oldStreamingLabelKeepMaxSecond; + private int oldLabelNumThreshold; @Before public void setUp() throws Exception { - + oldStreamingLabelKeepMaxSecond = Config.streaming_label_keep_max_second; + oldLabelNumThreshold = Config.label_num_threshold; } @After public void tearDown() throws Exception { + Config.streaming_label_keep_max_second = oldStreamingLabelKeepMaxSecond; + Config.label_num_threshold = oldLabelNumThreshold; File file = new File("./loadManagerTest"); if (file.exists()) { file.delete(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/CountStarSmallestSlotTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/CountStarSmallestSlotTest.java index c8dc09de9bdf65..d697cdf980086a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/CountStarSmallestSlotTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/CountStarSmallestSlotTest.java @@ -135,7 +135,6 @@ public void testGetSmallestSlotComplexTypePriority() { List slots = Lists.newArrayList(mapSlot, structSlot, arraySlot); SlotDescriptor result = PhysicalPlanTranslator.getSmallestSlot(slots); - // Should return the first one since they have same priority, compared by slot size Assertions.assertNotNull(result); Assertions.assertSame(structSlot, result); } @@ -433,6 +432,18 @@ public void testGetSmallestSlotAllTypesComprehensive() { Assertions.assertNotNull(false); } } + boolean hasMap = false; + boolean hasStruct = false; + boolean hasArray = false; + for (int i = 18; i <= 20; i++) { + PrimitiveType primitiveType = slots.get(i).getType().getPrimitiveType(); + hasMap = hasMap || primitiveType == PrimitiveType.MAP; + hasStruct = hasStruct || primitiveType == PrimitiveType.STRUCT; + hasArray = hasArray || primitiveType == PrimitiveType.ARRAY; + } + Assertions.assertTrue(hasMap); + Assertions.assertTrue(hasStruct); + Assertions.assertTrue(hasArray); } /** diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/ListPartitionPrunerV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/planner/ListPartitionPrunerV2Test.java index bb8693bc6a7040..2cb93df6b25683 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/ListPartitionPrunerV2Test.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/ListPartitionPrunerV2Test.java @@ -18,6 +18,7 @@ package org.apache.doris.planner; import org.apache.doris.analysis.PartitionValue; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.ListPartitionItem; import org.apache.doris.catalog.PartitionItem; import org.apache.doris.catalog.PartitionKey; @@ -26,17 +27,20 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.ThreadPoolManager; import org.apache.doris.common.security.authentication.ExecutionAuthenticator; +import org.apache.doris.datasource.CatalogMgr; import org.apache.doris.datasource.NameMapping; import org.apache.doris.datasource.hive.HMSCachedClient; import org.apache.doris.datasource.hive.HMSExternalCatalog; -import org.apache.doris.datasource.hive.HiveMetaStoreCache; -import org.apache.doris.datasource.hive.HiveMetaStoreCache.PartitionValueCacheKey; +import org.apache.doris.datasource.hive.HiveExternalMetaCache; +import org.apache.doris.datasource.hive.HiveExternalMetaCache.PartitionValueCacheKey; import org.apache.doris.datasource.hive.ThriftHMSCachedClient; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import mockit.Expectations; import mockit.Mock; import mockit.MockUp; +import mockit.Mocked; import org.apache.hadoop.hive.conf.HiveConf; import org.junit.Assert; import org.junit.Test; @@ -72,7 +76,25 @@ public void testPartitionValuesMap() throws AnalysisException { } @Test - public void testInvalidateTable() { + public void testInvalidateTable(@Mocked Env env, @Mocked CatalogMgr catalogMgr, + @Mocked HMSExternalCatalog hmsCatalog) { + long catalogId = 10001L; + + new Expectations() { + { + Env.getCurrentEnv(); + minTimes = 0; + result = env; + + env.getCatalogMgr(); + minTimes = 0; + result = catalogMgr; + + catalogMgr.getCatalog(catalogId); + minTimes = 0; + result = hmsCatalog; + } + }; new MockUp(HMSExternalCatalog.class) { @Mock @@ -96,8 +118,8 @@ public List listPartitionNames(String dbName, String tblName) { 10, 10, "mgr", 120, false); ThreadPoolExecutor listExecutor = ThreadPoolManager.newDaemonFixedThreadPool( 10, 10, "mgr", 120, false); - HiveMetaStoreCache cache = new HiveMetaStoreCache( - new HMSExternalCatalog(1L, "catalog", null, new HashMap<>(), null), executor, listExecutor); + HiveExternalMetaCache cache = new HiveExternalMetaCache(executor, listExecutor); + cache.initCatalog(catalogId, new HashMap<>()); ArrayList types = new ArrayList<>(); types.add(ScalarType.DOUBLE); @@ -105,11 +127,11 @@ public List listPartitionNames(String dbName, String tblName) { // the original partition of the table (in mock) will be loaded here String dbName = "db"; String tblName = "tb"; - PartitionValueCacheKey key = new PartitionValueCacheKey(NameMapping.createForTest("db", "tb"), types); - HiveMetaStoreCache.HivePartitionValues partitionValues = cache.getPartitionValues(key); + NameMapping nameMapping = NameMapping.createForTest(catalogId, dbName, tblName); + PartitionValueCacheKey key = new PartitionValueCacheKey(nameMapping, types); + HiveExternalMetaCache.HivePartitionValues partitionValues = cache.getPartitionValues(key); Assert.assertEquals(1, partitionValues.getIdToPartitionItem().size()); - Assert.assertTrue(partitionValues.getIdToPartitionItem().containsKey(8882801933302843777L)); - List items = partitionValues.getIdToPartitionItem().get(8882801933302843777L).getItems(); + List items = partitionValues.getIdToPartitionItem().values().iterator().next().getItems(); Assert.assertEquals(1, items.size()); PartitionKey partitionKey = items.get(0); Assert.assertEquals("1.234", partitionKey.getKeys().get(0).toString()); @@ -118,25 +140,30 @@ public List listPartitionNames(String dbName, String tblName) { // test add cache ArrayList values = new ArrayList<>(); values.add("c1=5.678000"); - cache.addPartitionsCache(NameMapping.createForTest(dbName, tblName), values, types); - HiveMetaStoreCache.HivePartitionValues partitionValues2 = cache.getPartitionValues( - new PartitionValueCacheKey(NameMapping.createForTest(dbName, tblName), types)); + cache.addPartitionsCache(nameMapping, values, types); + HiveExternalMetaCache.HivePartitionValues partitionValues2 = cache.getPartitionValues( + new PartitionValueCacheKey(nameMapping, types)); Assert.assertEquals(2, partitionValues2.getIdToPartitionItem().size()); - Assert.assertTrue(partitionValues2.getIdToPartitionItem().containsKey(7070400225537799947L)); - List items2 = partitionValues2.getIdToPartitionItem().get(7070400225537799947L).getItems(); - Assert.assertEquals(1, items2.size()); - PartitionKey partitionKey2 = items2.get(0); + PartitionKey partitionKey2 = null; + for (PartitionItem partitionItem : partitionValues2.getIdToPartitionItem().values()) { + List partitionKeys = partitionItem.getItems(); + Assert.assertEquals(1, partitionKeys.size()); + if ("5.678000".equals(partitionKeys.get(0).getOriginHiveKeys().get(0))) { + partitionKey2 = partitionKeys.get(0); + break; + } + } + Assert.assertNotNull(partitionKey2); Assert.assertEquals("5.678", partitionKey2.getKeys().get(0).toString()); Assert.assertEquals("5.678000", partitionKey2.getOriginHiveKeys().get(0)); // test refresh table // simulates the manually added partition table being deleted, leaving only one original partition in mock - cache.invalidateTableCache(NameMapping.createForTest(dbName, tblName)); - HiveMetaStoreCache.HivePartitionValues partitionValues3 = cache.getPartitionValues( - new PartitionValueCacheKey(NameMapping.createForTest(dbName, tblName), types)); + cache.invalidateTableCache(nameMapping); + HiveExternalMetaCache.HivePartitionValues partitionValues3 = cache.getPartitionValues( + new PartitionValueCacheKey(nameMapping, types)); Assert.assertEquals(1, partitionValues3.getIdToPartitionItem().size()); - Assert.assertTrue(partitionValues3.getIdToPartitionItem().containsKey(8882801933302843777L)); - List items3 = partitionValues3.getIdToPartitionItem().get(8882801933302843777L).getItems(); + List items3 = partitionValues3.getIdToPartitionItem().values().iterator().next().getItems(); Assert.assertEquals(1, items3.size()); PartitionKey partitionKey3 = items3.get(0); Assert.assertEquals("1.234", partitionKey3.getKeys().get(0).toString()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/HmsQueryCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/HmsQueryCacheTest.java index be7137f7b9950b..9c3274b00e8d29 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/qe/HmsQueryCacheTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/qe/HmsQueryCacheTest.java @@ -28,7 +28,7 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.common.jmockit.Deencapsulation; import org.apache.doris.datasource.CatalogMgr; -import org.apache.doris.datasource.ExternalSchemaCache; +import org.apache.doris.datasource.SchemaCacheKey; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalDatabase; import org.apache.doris.datasource.hive.HMSExternalTable; @@ -155,7 +155,7 @@ private void init(HMSExternalCatalog hmsCatalog) { Mockito.when(tbl.getDatabase()).thenReturn(db); Mockito.when(tbl.getUpdateTime()).thenReturn(NOW); // mock initSchemaAndUpdateTime and do nothing - Mockito.when(tbl.initSchema(Mockito.any(ExternalSchemaCache.SchemaCacheKey.class))) + Mockito.when(tbl.initSchema(Mockito.any(SchemaCacheKey.class))) .thenReturn(Optional.empty()); setField(tbl2, "objectCreated", true); @@ -179,7 +179,7 @@ private void init(HMSExternalCatalog hmsCatalog) { Mockito.when(tbl2.getUpdateTime()).thenReturn(NOW); Mockito.when(tbl2.getUpdateTime()).thenReturn(NOW); // mock initSchemaAndUpdateTime and do nothing - Mockito.when(tbl2.initSchemaAndUpdateTime(Mockito.any(ExternalSchemaCache.SchemaCacheKey.class))) + Mockito.when(tbl2.initSchemaAndUpdateTime(Mockito.any(SchemaCacheKey.class))) .thenReturn(Optional.empty()); Mockito.doNothing().when(tbl2).setUpdateTime(Mockito.anyLong()); @@ -250,7 +250,7 @@ public void testHitSqlCacheByNereidsAfterPartitionUpdateTimeChanged() { List scanNodes = Arrays.asList(hiveScanNode4); // invoke initSchemaAndUpdateTime first and init schemaUpdateTime - tbl2.initSchemaAndUpdateTime(new ExternalSchemaCache.SchemaCacheKey(tbl2.getOrBuildNameMapping())); + tbl2.initSchemaAndUpdateTime(new SchemaCacheKey(tbl2.getOrBuildNameMapping())); CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheModeForNereids(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); 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 2553df10bb36b8..9332744448aedd 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 @@ -200,7 +200,16 @@ suite("test_external_catalog_hive", "p0,external,hive,external_docker,external_d } // 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}";""" + sql """ + select catalog_name, engine_name, entry_name, request_count, hit_count, miss_count, load_failure_count + from internal.information_schema.catalog_meta_cache_statistics + order by catalog_name, engine_name, entry_name; + """ + sql """ + select catalog_name, engine_name, entry_name, request_count, hit_count, miss_count, load_failure_count + from ${catalog_name}.information_schema.catalog_meta_cache_statistics + where catalog_name="${catalog_name}" + order by catalog_name, engine_name, entry_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 710a0bc953f9c4..938e0ce658e8ab 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 @@ -60,11 +60,19 @@ suite("test_iceberg_table_stats", "p0,external,doris,external_docker,external_do 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}";""" + sql """ + select catalog_name, engine_name, entry_name, request_count, hit_count, miss_count, load_failure_count + from information_schema.catalog_meta_cache_statistics + order by catalog_name, engine_name, entry_name; + """ + sql """ + select catalog_name, engine_name, entry_name, request_count, hit_count, miss_count, load_failure_count + from information_schema.catalog_meta_cache_statistics + where catalog_name="${catalog_name}" + order by catalog_name, engine_name, entry_name; + """ } finally { } } } - From 04baf35ba816e12ec6c43df766ad1d30252195d3 Mon Sep 17 00:00:00 2001 From: Socrates Date: Thu, 26 Mar 2026 11:58:56 +0800 Subject: [PATCH 2/3] [fix](catalog) Fix CatalogMgr import order --- .../src/main/java/org/apache/doris/datasource/CatalogMgr.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java index 9212761ebcd26f..3dbe0e493d6a22 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java @@ -22,8 +22,8 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.EnvFactory; import org.apache.doris.catalog.TableIf; -import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.catalog.Type; +import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.CaseSensibility; import org.apache.doris.common.DdlException; From bbe15caacfc7a9516dd40277f12aa49a8057ef3b Mon Sep 17 00:00:00 2001 From: Socrates Date: Thu, 26 Mar 2026 14:13:51 +0800 Subject: [PATCH 3/3] [fix](catalog) Remove unsupported constraint cleanup on branch-4.1 --- .../src/main/java/org/apache/doris/datasource/CatalogMgr.java | 1 - 1 file changed, 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java index 3dbe0e493d6a22..21c6320a155ba7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java @@ -143,7 +143,6 @@ private void cleanupRemovedCatalog(RemovedCatalog removedCatalog) { } CatalogIf catalog = removedCatalog.catalog; catalog.onClose(); - Env.getCurrentEnv().getConstraintManager().dropCatalogConstraints(removedCatalog.catalogName); ConnectContext ctx = ConnectContext.get(); if (ctx != null) { ctx.removeLastDBOfCatalog(removedCatalog.catalogName);