diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run08.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run08.sql new file mode 100644 index 00000000000000..aa573e0af1e9fb --- /dev/null +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run08.sql @@ -0,0 +1,106 @@ + +use demo.test_db; +CREATE TABLE no_partition ( + id INT, + name STRING, + create_date DATE +) USING iceberg; +INSERT INTO no_partition VALUES(1, 'Alice', DATE '2023-12-01'),(2, 'Bob', DATE '2023-12-02'); + +CREATE TABLE not_support_trans ( + id INT, + name STRING, + create_date DATE +) USING iceberg +PARTITIONED BY (bucket(10, create_date)); +INSERT INTO not_support_trans VALUES(1, 'Alice', DATE '2023-12-01'),(2, 'Bob', DATE '2023-12-02'); + +CREATE TABLE add_partition1 ( + id INT, + name STRING, + create_date DATE +) USING iceberg; +INSERT INTO add_partition1 VALUES(1, 'Alice', DATE '2023-12-01'),(2, 'Bob', DATE '2023-12-02'); +ALTER TABLE add_partition1 ADD PARTITION FIELD month(create_date); +INSERT INTO add_partition1 VALUES(3, 'Lara', DATE '2023-12-03'); + +CREATE TABLE add_partition2 ( + id INT, + name STRING, + create_date1 DATE, + create_date2 DATE +) USING iceberg +PARTITIONED BY (month(create_date1)); +INSERT INTO add_partition2 VALUES(1, 'Alice', DATE '2023-12-01', DATE '2023-12-01'),(2, 'Bob', DATE '2023-12-02', DATE '2023-12-02'); +ALTER TABLE add_partition2 ADD PARTITION FIELD year(create_date2); +INSERT INTO add_partition2 VALUES(3, 'Lara', DATE '2023-12-03', DATE '2023-12-03'); + +CREATE TABLE drop_partition1 ( + id INT, + name STRING, + create_date DATE +) USING iceberg +PARTITIONED BY (month(create_date)); +INSERT INTO drop_partition1 VALUES(1, 'Alice', DATE '2023-12-01'),(2, 'Bob', DATE '2023-12-02'); +ALTER TABLE drop_partition1 DROP PARTITION FIELD month(create_date); + +CREATE TABLE drop_partition2 ( + id INT, + name STRING, + create_date1 DATE, + create_date2 DATE +) USING iceberg +PARTITIONED BY (month(create_date1), year(create_date2)); +INSERT INTO drop_partition2 VALUES(1, 'Alice', DATE '2023-12-01', DATE '2023-12-01'),(2, 'Bob', DATE '2023-12-02', DATE '2023-12-02'); +ALTER TABLE drop_partition2 DROP PARTITION FIELD year(create_date2); +INSERT INTO drop_partition2 VALUES(3, 'Lara', DATE '2023-12-03', DATE '2023-12-03'); + +CREATE TABLE replace_partition1 ( + id INT, + name STRING, + create_date1 DATE, + create_date2 DATE +) USING iceberg +PARTITIONED BY (month(create_date1)); +INSERT INTO replace_partition1 VALUES(1, 'Alice', DATE '2023-12-01', DATE '2023-12-01'),(2, 'Bob', DATE '2023-12-02', DATE '2023-12-02'); +ALTER TABLE replace_partition1 REPLACE PARTITION FIELD month(create_date1) WITH year(create_date2); +INSERT INTO replace_partition1 VALUES(3, 'Lara', DATE '2023-12-03', DATE '2023-12-03'); + +CREATE TABLE replace_partition2( + ts TIMESTAMP COMMENT 'ts', + value INT COMMENT 'col1') +USING iceberg +PARTITIONED BY (month(ts)); +insert into replace_partition2 values (to_timestamp('2024-10-26 11:02:03', 'yyyy-MM-dd HH:mm:ss'), 1), (to_timestamp('2024-11-27 21:02:03', 'yyyy-MM-dd HH:mm:ss'), 2); +ALTER TABLE replace_partition2 REPLACE PARTITION FIELD ts_month WITH day(ts); +insert into replace_partition2 values (to_timestamp('2024-12-03 14:02:03', 'yyyy-MM-dd HH:mm:ss'), 3); + +CREATE TABLE replace_partition3( + ts TIMESTAMP COMMENT 'ts', + value INT COMMENT 'col1') +USING iceberg +PARTITIONED BY (month(ts)); +insert into replace_partition3 values (to_timestamp('2024-11-26 11:02:03', 'yyyy-MM-dd HH:mm:ss'), 1); +ALTER TABLE replace_partition3 REPLACE PARTITION FIELD month(ts) WITH day(ts); +insert into replace_partition3 values (to_timestamp('2024-11-02 21:02:03', 'yyyy-MM-dd HH:mm:ss'), 2), (to_timestamp('2024-11-03 11:02:03', 'yyyy-MM-dd HH:mm:ss'), 3), (to_timestamp('2024-12-02 19:02:03', 'yyyy-MM-dd HH:mm:ss'), 4); + +CREATE TABLE replace_partition4( + ts TIMESTAMP COMMENT 'ts', + value INT COMMENT 'col1') +USING iceberg +PARTITIONED BY (month(ts)); +insert into replace_partition4 values (to_timestamp('2024-10-26 11:02:03', 'yyyy-MM-dd HH:mm:ss'), 1), (to_timestamp('2024-11-26 21:02:03', 'yyyy-MM-dd HH:mm:ss'), 2); +ALTER TABLE replace_partition4 REPLACE PARTITION FIELD month(ts) WITH day(ts); +insert into replace_partition4 values (to_timestamp('2024-11-02 13:02:03', 'yyyy-MM-dd HH:mm:ss'), 3), (to_timestamp('2024-11-03 10:02:03', 'yyyy-MM-dd HH:mm:ss'), 4); + +CREATE TABLE replace_partition5( + ts TIMESTAMP COMMENT 'ts', + value INT COMMENT 'col1') +USING iceberg +PARTITIONED BY (month(ts)); +insert into replace_partition5 values (to_timestamp('2024-10-26 11:02:03', 'yyyy-MM-dd HH:mm:ss'), 1), (to_timestamp('2024-11-26 13:02:03', 'yyyy-MM-dd HH:mm:ss'), 2); +ALTER TABLE replace_partition5 REPLACE PARTITION FIELD month(ts) WITH day(ts); +insert into replace_partition5 values (to_timestamp('2024-10-12 09:02:03', 'yyyy-MM-dd HH:mm:ss'), 3), (to_timestamp('2024-12-21 21:02:03', 'yyyy-MM-dd HH:mm:ss'), 4); +ALTER TABLE replace_partition5 REPLACE PARTITION FIELD day(ts) WITH hour(ts); +insert into replace_partition5 values (to_timestamp('2024-12-21 11:02:03', 'yyyy-MM-dd HH:mm:ss'), 5); +insert into replace_partition5 values (to_timestamp('2025-01-01 11:02:03', 'yyyy-MM-dd HH:mm:ss'), 6); diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/paimon/run01.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/paimon/run01.sql index 7722d09463686e..13fd983744a22f 100644 --- a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/paimon/run01.sql +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/paimon/run01.sql @@ -4,9 +4,9 @@ use test_paimon_spark; drop table if exists test_tb_mix_format; create table test_tb_mix_format ( - id int, - value int, - par string + id int, + value int, + par string ) PARTITIONED BY (par) TBLPROPERTIES ( 'primary-key' = 'id, par', 'bucket'=1000, @@ -26,9 +26,9 @@ alter table test_tb_mix_format unset TBLPROPERTIES ('file.format'); drop table if exists two_partition; CREATE TABLE two_partition ( - id BIGINT, - create_date STRING, - region STRING + id BIGINT, + create_date STRING, + region STRING ) PARTITIONED BY (create_date,region) TBLPROPERTIES ( 'primary-key' = 'create_date,region,id', 'bucket'=10, @@ -43,8 +43,8 @@ insert into two_partition values(5,'2038-01-02','bj'); drop table if exists null_partition; CREATE TABLE null_partition ( - id BIGINT, - region STRING + id BIGINT, + region STRING ) PARTITIONED BY (region) TBLPROPERTIES ( 'primary-key' = 'region,id', 'bucket'=10, @@ -59,8 +59,8 @@ insert into null_partition values(5,'NULL'); drop table if exists date_partition; CREATE TABLE date_partition ( - id BIGINT, - create_date DATE + id BIGINT, + create_date DATE ) PARTITIONED BY (create_date) TBLPROPERTIES ( 'primary-key' = 'create_date,id', 'bucket'=10, diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionsStmt.java index 0be41ef60fa7ba..14a00184c755d4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionsStmt.java @@ -37,6 +37,8 @@ import org.apache.doris.datasource.CatalogIf; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalTable; +import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; +import org.apache.doris.datasource.iceberg.IcebergExternalTable; import org.apache.doris.datasource.maxcompute.MaxComputeExternalCatalog; import org.apache.doris.datasource.maxcompute.MaxComputeExternalTable; import org.apache.doris.mysql.privilege.PrivPredicate; @@ -128,7 +130,8 @@ public void analyze(Analyzer analyzer) throws UserException { DatabaseIf db = catalog.getDbOrAnalysisException(dbName); TableIf table = db.getTableOrMetaException(tblName, Table.TableType.OLAP, - TableType.HMS_EXTERNAL_TABLE, TableType.MAX_COMPUTE_EXTERNAL_TABLE); + TableType.HMS_EXTERNAL_TABLE, TableType.MAX_COMPUTE_EXTERNAL_TABLE, + TableType.ICEBERG_EXTERNAL_TABLE); if (table instanceof HMSExternalTable) { if (((HMSExternalTable) table).isView()) { @@ -147,6 +150,13 @@ public void analyze(Analyzer analyzer) throws UserException { return; } + if (table instanceof IcebergExternalTable) { + if (!((IcebergExternalTable) table).isValidRelatedTable()) { + throw new AnalysisException("Table " + tblName + " is not a supported partition table"); + } + return; + } + table.readLock(); try { // build proc path @@ -180,7 +190,7 @@ public void analyzeImpl(Analyzer analyzer) throws UserException { // disallow unsupported catalog if (!(catalog.isInternalCatalog() || catalog instanceof HMSExternalCatalog - || catalog instanceof MaxComputeExternalCatalog)) { + || catalog instanceof MaxComputeExternalCatalog || catalog instanceof IcebergExternalCatalog)) { throw new AnalysisException(String.format("Catalog of type '%s' is not allowed in ShowPartitionsStmt", catalog.getType())); } @@ -287,6 +297,10 @@ public ShowResultSetMetaData getMetaData() { for (String col : result.getColumnNames()) { builder.addColumn(new Column(col, ScalarType.createVarchar(30))); } + } else if (catalog instanceof IcebergExternalCatalog) { + builder.addColumn(new Column("Partition", ScalarType.createVarchar(60))); + builder.addColumn(new Column("Lower Bound", ScalarType.createVarchar(100))); + builder.addColumn(new Column("Upper Bound", ScalarType.createVarchar(100))); } else { builder.addColumn(new Column("Partition", ScalarType.createVarchar(60))); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java index 2084703989d42c..337ad38bbf9b89 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java @@ -361,7 +361,7 @@ public MTMVRefreshSnapshot getRefreshSnapshot() { * * @return mvPartitionName ==> mvPartitionKeyDesc */ - public Map generateMvPartitionDescs() { + public Map generateMvPartitionDescs() throws AnalysisException { Map mtmvItems = getAndCopyPartitionItems(); Map result = Maps.newHashMap(); for (Entry entry : mtmvItems.entrySet()) { 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 79dd86345eebfc..9b16385e449e7b 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 @@ -34,6 +34,7 @@ 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.paimon.PaimonMetadataCacheMgr; import org.apache.doris.fs.FileSystemCache; @@ -293,7 +294,7 @@ public void addPartitionsCache(long catalogId, HMSExternalTable table, List partitionColumnTypes; try { - partitionColumnTypes = table.getPartitionColumnTypes(); + partitionColumnTypes = table.getPartitionColumnTypes(MvccUtil.getSnapshotFromContext(table)); } catch (NotSupportedException e) { LOG.warn("Ignore not supported hms table, message: {} ", e.getMessage()); return; 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 30bf48c3d8b76a..60a1f172978124 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 @@ -391,7 +391,7 @@ public List getChunkSizes() { throw new NotImplementedException("getChunkSized not implemented"); } - protected Optional getSchemaCacheValue() { + public Optional getSchemaCacheValue() { ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); return cache.getSchemaValue(dbName, name); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/TablePartitionValues.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/TablePartitionValues.java index c7f2ce6f712d6b..e928d3c739eed3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/TablePartitionValues.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/TablePartitionValues.java @@ -51,6 +51,7 @@ public class TablePartitionValues { private long nextPartitionId; private final Map idToPartitionItem; private final Map partitionNameToIdMap; + private Map partitionNameToLastModifiedMap; private final Map partitionIdToNameMap; private Map> idToUniqueIdsMap; @@ -68,15 +69,12 @@ public TablePartitionValues() { nextPartitionId = 0; idToPartitionItem = new HashMap<>(); partitionNameToIdMap = new HashMap<>(); + partitionNameToLastModifiedMap = new HashMap<>(); partitionIdToNameMap = new HashMap<>(); } - public TablePartitionValues(List partitionNames, List> partitionValues, List types) { - this(); - addPartitions(partitionNames, partitionValues, types); - } - - public void addPartitions(List partitionNames, List> partitionValues, List types) { + public void addPartitions(List partitionNames, List> partitionValues, List types, + List partitionLastUpdateTimestamp) { Preconditions.checkState(partitionNames.size() == partitionValues.size()); List addPartitionNames = new ArrayList<>(); List addPartitionItems = new ArrayList<>(); @@ -90,6 +88,7 @@ public void addPartitions(List partitionNames, List> partit addPartitionNames.add(partitionNames.get(i)); addPartitionItems.add(toListPartitionItem(partitionValues.get(i), types)); } + partitionNameToLastModifiedMap.put(partitionNames.get(i), partitionLastUpdateTimestamp.get(i)); } cleanPartitions(); @@ -123,23 +122,6 @@ private void addPartitionItems(List partitionNames, List partitionValuesMap = ListPartitionPrunerV2.getPartitionValuesMap(idToPartitionItem); } - public void dropPartitions(List partitionNames, List types) { - partitionNames.forEach(p -> { - Long removedPartition = partitionNameToIdMap.get(p); - if (removedPartition != null) { - idToPartitionItem.remove(removedPartition); - } - }); - List remainingPartitionNames = new ArrayList<>(); - List remainingPartitionItems = new ArrayList<>(); - partitionNameToIdMap.forEach((partitionName, partitionId) -> { - remainingPartitionNames.add(partitionName); - remainingPartitionItems.add(idToPartitionItem.get(partitionId)); - }); - cleanPartitions(); - addPartitionItems(remainingPartitionNames, remainingPartitionItems, types); - } - public long getLastUpdateTimestamp() { return lastUpdateTimestamp; } @@ -148,6 +130,10 @@ public void setLastUpdateTimestamp(long lastUpdateTimestamp) { this.lastUpdateTimestamp = lastUpdateTimestamp; } + public Map getPartitionNameToLastModifiedMap() { + return partitionNameToLastModifiedMap; + } + public Lock readLock() { return readWriteLock.readLock(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSDlaTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSDlaTable.java new file mode 100644 index 00000000000000..7894279b2950ac --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSDlaTable.java @@ -0,0 +1,76 @@ +// 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.catalog.Column; +import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.catalog.PartitionType; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; +import org.apache.doris.datasource.mvcc.MvccSnapshot; +import org.apache.doris.mtmv.MTMVBaseTableIf; +import org.apache.doris.mtmv.MTMVRefreshContext; +import org.apache.doris.mtmv.MTMVSnapshotIf; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * This abstract class represents a Hive Metastore (HMS) Dla Table and provides a blueprint for + * various operations related to metastore tables in Doris. + * + * Purpose: + * - To encapsulate common functionalities that HMS Dla tables should have for implementing other interfaces + * + * Why needed: + * - To provide a unified way to manage and interact with different kinds of Dla Table + * - To facilitate the implementation of multi-table materialized views (MTMV) by providing necessary + * methods for snapshot and partition management. + * - To abstract out the specific details of HMS table operations, making the code more modular and maintainable. + */ +public abstract class HMSDlaTable implements MTMVBaseTableIf { + protected HMSExternalTable hmsTable; + + public HMSDlaTable(HMSExternalTable table) { + this.hmsTable = table; + } + + abstract Map getAndCopyPartitionItems(Optional snapshot) + throws AnalysisException; + + abstract PartitionType getPartitionType(Optional snapshot); + + abstract Set getPartitionColumnNames(Optional snapshot); + + abstract List getPartitionColumns(Optional snapshot); + + abstract MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, + Optional snapshot) throws AnalysisException; + + abstract MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) + throws AnalysisException; + + abstract boolean isPartitionColumnAllowNull(); + + @Override + public void beforeMTMVRefresh(MTMV mtmv) throws DdlException { + } +} 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 72d52dc13612cb..f2097a0bf01522 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 @@ -31,22 +31,27 @@ 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.SchemaCacheValue; import org.apache.doris.datasource.TablePartitionValues; +import org.apache.doris.datasource.hudi.HudiMvccSnapshot; +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.IcebergSchemaCacheKey; import org.apache.doris.datasource.iceberg.IcebergUtils; +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.SupportedSysTables; import org.apache.doris.datasource.systable.SysTable; import org.apache.doris.mtmv.MTMVBaseTableIf; -import org.apache.doris.mtmv.MTMVMaxTimestampSnapshot; import org.apache.doris.mtmv.MTMVRefreshContext; import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.mtmv.MTMVSnapshotIf; -import org.apache.doris.mtmv.MTMVTimestampSnapshot; import org.apache.doris.nereids.exceptions.NotSupportedException; import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan.SelectedPartitions; import org.apache.doris.qe.GlobalVariable; @@ -80,6 +85,9 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Types; +import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -100,7 +108,7 @@ /** * Hive metastore external table. */ -public class HMSExternalTable extends ExternalTable implements MTMVRelatedTableIf, MTMVBaseTableIf { +public class HMSExternalTable extends ExternalTable implements MTMVRelatedTableIf, MTMVBaseTableIf, MvccTable { private static final Logger LOG = LogManager.getLogger(HMSExternalTable.class); public static final Set SUPPORTED_HIVE_FILE_FORMATS; @@ -162,6 +170,8 @@ public class HMSExternalTable extends ExternalTable implements MTMVRelatedTableI private DLAType dlaType = DLAType.UNKNOWN; + private HMSDlaTable dlaTable; + // record the event update time when enable hms event listener protected volatile long eventUpdateTime; @@ -199,10 +209,13 @@ protected synchronized void makeSureInitialized() { } else { if (supportedIcebergTable()) { dlaType = DLAType.ICEBERG; + dlaTable = new HiveDlaTable(this); } else if (supportedHoodieTable()) { dlaType = DLAType.HUDI; + dlaTable = new HudiDlaTable(this); } else if (supportedHiveTable()) { dlaType = DLAType.HIVE; + dlaTable = new HiveDlaTable(this); } else { // Should not reach here. Because `supportedHiveTable` will throw exception if not return true. throw new NotSupportedException("Unsupported dlaType for table: " + getNameWithFullQualifiers()); @@ -293,23 +306,45 @@ public org.apache.hadoop.hive.metastore.api.Table getRemoteTable() { return remoteTable; } - public List getPartitionColumnTypes() { + @Override + public List getFullSchema() { makeSureInitialized(); + ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); + if (getDlaType() == DLAType.HUDI) { + return ((HudiDlaTable) dlaTable).getHudiSchemaCacheValue(MvccUtil.getSnapshotFromContext(this)) + .getSchema(); + } + Optional schemaCacheValue = cache.getSchemaValue(dbName, name); + return schemaCacheValue.map(SchemaCacheValue::getSchema).orElse(null); + } + + public List getPartitionColumnTypes(Optional snapshot) { + makeSureInitialized(); + if (getDlaType() == DLAType.HUDI) { + return ((HudiDlaTable) dlaTable).getHudiSchemaCacheValue(snapshot).getPartitionColTypes(); + } Optional schemaCacheValue = getSchemaCacheValue(); return schemaCacheValue.map(value -> ((HMSSchemaCacheValue) value).getPartitionColTypes()) .orElse(Collections.emptyList()); } - public List getPartitionColumns() { + public List getHudiPartitionColumnTypes(long timestamp) { makeSureInitialized(); - Optional schemaCacheValue = getSchemaCacheValue(); - return schemaCacheValue.map(value -> ((HMSSchemaCacheValue) value).getPartitionColumns()) + ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); + Optional schemaCacheValue = cache.getSchemaValue( + new HudiSchemaCacheKey(dbName, name, timestamp)); + return schemaCacheValue.map(value -> ((HMSSchemaCacheValue) value).getPartitionColTypes()) .orElse(Collections.emptyList()); } + public List getPartitionColumns() { + return getPartitionColumns(MvccUtil.getSnapshotFromContext(this)); + } + @Override public List getPartitionColumns(Optional snapshot) { - return getPartitionColumns(); + makeSureInitialized(); + return dlaTable.getPartitionColumns(snapshot); } @Override @@ -349,7 +384,7 @@ public Map getNameToPartitionItems() { } HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() .getMetaStoreCache((HMSExternalCatalog) this.getCatalog()); - List partitionColumnTypes = this.getPartitionColumnTypes(); + List partitionColumnTypes = this.getPartitionColumnTypes(MvccUtil.getSnapshotFromContext(this)); HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( this.getDbName(), this.getName(), partitionColumnTypes); Map idToPartitionItem = hivePartitionValues.getIdToPartitionItem(); @@ -516,10 +551,6 @@ public Set getPartitionNames() { @Override public Optional initSchemaAndUpdateTime(SchemaCacheKey key) { - return initSchemaAndUpdateTime(); - } - - public Optional initSchemaAndUpdateTime() { org.apache.hadoop.hive.metastore.api.Table table = ((HMSExternalCatalog) catalog).getClient() .getTable(dbName, name); // try to use transient_lastDdlTime from hms client @@ -528,7 +559,7 @@ public Optional initSchemaAndUpdateTime() { ? Long.parseLong(table.getParameters().get(TBL_PROP_TRANSIENT_LAST_DDL_TIME)) * 1000 // use current timestamp if lastDdlTime does not exist (hive views don't have this prop) : System.currentTimeMillis(); - return initSchema(); + return initSchema(key); } public long getLastDdlTime() { @@ -541,35 +572,41 @@ public long getLastDdlTime() { } @Override - public Optional initSchema() { + public Optional initSchema(SchemaCacheKey key) { makeSureInitialized(); if (dlaType.equals(DLAType.ICEBERG)) { - return getIcebergSchema(); + return getIcebergSchema(key); } else if (dlaType.equals(DLAType.HUDI)) { - return getHudiSchema(); + return getHudiSchema(key); } else { return getHiveSchema(); } } - private Optional getIcebergSchema() { - List columns = IcebergUtils.getSchema(catalog, dbName, name); - List partitionColumns = initPartitionColumns(columns); - return Optional.of(new HMSSchemaCacheValue(columns, partitionColumns)); + private Optional getIcebergSchema(SchemaCacheKey key) { + return IcebergUtils.loadSchemaCacheValue(catalog, dbName, name, ((IcebergSchemaCacheKey) key).getSchemaId()); } - private Optional getHudiSchema() { - org.apache.avro.Schema hudiSchema = HiveMetaStoreClientHelper.getHudiTableSchema(this); + private Optional getHudiSchema(SchemaCacheKey key) { + boolean[] enableSchemaEvolution = {false}; + HudiSchemaCacheKey hudiSchemaCacheKey = (HudiSchemaCacheKey) key; + InternalSchema hudiInternalSchema = HiveMetaStoreClientHelper.getHudiTableSchema(this, enableSchemaEvolution, + Long.toString(hudiSchemaCacheKey.getTimestamp())); + org.apache.avro.Schema hudiSchema = AvroInternalSchemaConverter.convert(hudiInternalSchema, name); List tmpSchema = Lists.newArrayListWithCapacity(hudiSchema.getFields().size()); List colTypes = Lists.newArrayList(); - for (org.apache.avro.Schema.Field hudiField : hudiSchema.getFields()) { - String columnName = hudiField.name().toLowerCase(Locale.ROOT); - tmpSchema.add(new Column(columnName, HudiUtils.fromAvroHudiTypeToDorisType(hudiField.schema()), - true, null, true, null, "", true, null, -1, null)); - colTypes.add(HudiUtils.convertAvroToHiveType(hudiField.schema())); + for (int i = 0; i < hudiSchema.getFields().size(); i++) { + Types.Field hudiInternalfield = hudiInternalSchema.getRecord().fields().get(i); + org.apache.avro.Schema.Field hudiAvroField = hudiSchema.getFields().get(i); + String columnName = hudiAvroField.name().toLowerCase(Locale.ROOT); + tmpSchema.add(new Column(columnName, HudiUtils.fromAvroHudiTypeToDorisType(hudiAvroField.schema()), + true, null, true, null, "", true, null, + hudiInternalfield.fieldId(), null)); + colTypes.add(HudiUtils.convertAvroToHiveType(hudiAvroField.schema())); } List partitionColumns = initPartitionColumns(tmpSchema); - HudiSchemaCacheValue hudiSchemaCacheValue = new HudiSchemaCacheValue(tmpSchema, partitionColumns); + HudiSchemaCacheValue hudiSchemaCacheValue = + new HudiSchemaCacheValue(tmpSchema, partitionColumns, enableSchemaEvolution[0]); hudiSchemaCacheValue.setColTypes(colTypes); return Optional.of(hudiSchemaCacheValue); } @@ -819,96 +856,46 @@ public Set getDistributionColumnNames() { @Override public PartitionType getPartitionType(Optional snapshot) { - return getPartitionType(); + makeSureInitialized(); + return dlaTable.getPartitionType(snapshot); } - public PartitionType getPartitionType() { - return getPartitionColumns().size() > 0 ? PartitionType.LIST : PartitionType.UNPARTITIONED; + public Set getPartitionColumnNames() { + return getPartitionColumnNames(MvccUtil.getSnapshotFromContext(this)); } @Override public Set getPartitionColumnNames(Optional snapshot) { - return getPartitionColumnNames(); - } - - public Set getPartitionColumnNames() { - return getPartitionColumns().stream() - .map(c -> c.getName().toLowerCase()).collect(Collectors.toSet()); + makeSureInitialized(); + return dlaTable.getPartitionColumnNames(snapshot); } @Override - public Map getAndCopyPartitionItems(Optional snapshot) { - return getNameToPartitionItems(); + public Map getAndCopyPartitionItems(Optional snapshot) + throws AnalysisException { + makeSureInitialized(); + return dlaTable.getAndCopyPartitionItems(snapshot); } @Override public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, Optional snapshot) throws AnalysisException { - HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getMetaStoreCache((HMSExternalCatalog) getCatalog()); - HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( - getDbName(), getName(), getPartitionColumnTypes()); - Long partitionId = getPartitionIdByNameOrAnalysisException(partitionName, hivePartitionValues); - HivePartition hivePartition = getHivePartitionByIdOrAnalysisException(partitionId, - hivePartitionValues, cache); - return new MTMVTimestampSnapshot(hivePartition.getLastModifiedTime()); + makeSureInitialized(); + return dlaTable.getPartitionSnapshot(partitionName, context, snapshot); } @Override public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) throws AnalysisException { - if (getPartitionType() == PartitionType.UNPARTITIONED) { - return new MTMVMaxTimestampSnapshot(getName(), getLastDdlTime()); - } - HivePartition maxPartition = null; - long maxVersionTime = 0L; - long visibleVersionTime; - HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getMetaStoreCache((HMSExternalCatalog) getCatalog()); - HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( - getDbName(), getName(), getPartitionColumnTypes()); - List partitionList = cache.getAllPartitionsWithCache(getDbName(), getName(), - Lists.newArrayList(hivePartitionValues.getPartitionValuesMap().values())); - if (CollectionUtils.isEmpty(partitionList)) { - throw new AnalysisException("partitionList is empty, table name: " + getName()); - } - for (HivePartition hivePartition : partitionList) { - visibleVersionTime = hivePartition.getLastModifiedTime(); - if (visibleVersionTime > maxVersionTime) { - maxVersionTime = visibleVersionTime; - maxPartition = hivePartition; - } - } - return new MTMVMaxTimestampSnapshot(maxPartition.getPartitionName(getPartitionColumns()), maxVersionTime); - } - - private Long getPartitionIdByNameOrAnalysisException(String partitionName, - HiveMetaStoreCache.HivePartitionValues hivePartitionValues) - throws AnalysisException { - Long partitionId = hivePartitionValues.getPartitionNameToIdMap().get(partitionName); - if (partitionId == null) { - throw new AnalysisException("can not find partition: " + partitionName); - } - return partitionId; + makeSureInitialized(); + return dlaTable.getTableSnapshot(context, snapshot); } - private HivePartition getHivePartitionByIdOrAnalysisException(Long partitionId, - HiveMetaStoreCache.HivePartitionValues hivePartitionValues, - HiveMetaStoreCache cache) throws AnalysisException { - List partitionValues = hivePartitionValues.getPartitionValuesMap().get(partitionId); - if (CollectionUtils.isEmpty(partitionValues)) { - throw new AnalysisException("can not find partitionValues: " + partitionId); - } - HivePartition partition = cache.getHivePartition(getDbName(), getName(), partitionValues); - if (partition == null) { - throw new AnalysisException("can not find partition: " + partitionId); - } - return partition; - } @Override public boolean isPartitionColumnAllowNull() { - return true; + makeSureInitialized(); + return dlaTable.isPartitionColumnAllowNull(); } /** @@ -970,7 +957,7 @@ private HiveMetaStoreCache.HivePartitionValues getAllPartitionValues() { } HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() .getMetaStoreCache((HMSExternalCatalog) catalog); - List partitionColumnTypes = getPartitionColumnTypes(); + List partitionColumnTypes = getPartitionColumnTypes(MvccUtil.getSnapshotFromContext(this)); HiveMetaStoreCache.HivePartitionValues partitionValues = null; // Get table partitions from cache. if (!partitionColumnTypes.isEmpty()) { @@ -1045,6 +1032,14 @@ public boolean isPartitionedTable() { public void beforeMTMVRefresh(MTMV mtmv) throws DdlException { } + @Override + public MvccSnapshot loadSnapshot(Optional tableSnapshot) { + if (getDlaType() == DLAType.HUDI) { + return new HudiMvccSnapshot(HudiUtils.getPartitionValues(tableSnapshot, this)); + } + return new EmptyMvccSnapshot(); + } + public boolean firstColumnIsString() { List columns = getColumns(); if (columns == null || columns.isEmpty()) { 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 new file mode 100644 index 00000000000000..296b2f3667aa37 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveDlaTable.java @@ -0,0 +1,141 @@ +// 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.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.catalog.PartitionType; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.doris.datasource.mvcc.MvccSnapshot; +import org.apache.doris.mtmv.MTMVMaxTimestampSnapshot; +import org.apache.doris.mtmv.MTMVRefreshContext; +import org.apache.doris.mtmv.MTMVSnapshotIf; +import org.apache.doris.mtmv.MTMVTimestampSnapshot; + +import com.google.common.collect.Lists; +import org.apache.commons.collections.CollectionUtils; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +public class HiveDlaTable extends HMSDlaTable { + + public HiveDlaTable(HMSExternalTable table) { + super(table); + } + + @Override + public PartitionType getPartitionType(Optional snapshot) { + return getPartitionColumns(snapshot).size() > 0 ? PartitionType.LIST : PartitionType.UNPARTITIONED; + } + + @Override + public Set getPartitionColumnNames(Optional snapshot) { + return getPartitionColumns(snapshot).stream() + .map(c -> c.getName().toLowerCase()).collect(Collectors.toSet()); + } + + @Override + public List getPartitionColumns(Optional snapshot) { + Optional schemaCacheValue = hmsTable.getSchemaCacheValue(); + return schemaCacheValue.map(value -> ((HMSSchemaCacheValue) value).getPartitionColumns()) + .orElse(Collections.emptyList()); + } + + @Override + public Map getAndCopyPartitionItems(Optional snapshot) { + return hmsTable.getNameToPartitionItems(); + } + + @Override + public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, + Optional snapshot) throws AnalysisException { + HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() + .getMetaStoreCache((HMSExternalCatalog) hmsTable.getCatalog()); + HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( + hmsTable.getDbName(), hmsTable.getName(), hmsTable.getPartitionColumnTypes(snapshot)); + Long partitionId = getPartitionIdByNameOrAnalysisException(partitionName, hivePartitionValues); + HivePartition hivePartition = getHivePartitionByIdOrAnalysisException(partitionId, + hivePartitionValues, cache); + return new MTMVTimestampSnapshot(hivePartition.getLastModifiedTime()); + } + + @Override + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) + throws AnalysisException { + if (hmsTable.getPartitionType(snapshot) == PartitionType.UNPARTITIONED) { + return new MTMVMaxTimestampSnapshot(hmsTable.getName(), hmsTable.getLastDdlTime()); + } + HivePartition maxPartition = null; + long maxVersionTime = 0L; + long visibleVersionTime; + HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() + .getMetaStoreCache((HMSExternalCatalog) hmsTable.getCatalog()); + HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( + hmsTable.getDbName(), hmsTable.getName(), hmsTable.getPartitionColumnTypes(snapshot)); + List partitionList = cache.getAllPartitionsWithCache(hmsTable.getDbName(), hmsTable.getName(), + Lists.newArrayList(hivePartitionValues.getPartitionValuesMap().values())); + if (CollectionUtils.isEmpty(partitionList)) { + throw new AnalysisException("partitionList is empty, table name: " + hmsTable.getName()); + } + for (HivePartition hivePartition : partitionList) { + visibleVersionTime = hivePartition.getLastModifiedTime(); + if (visibleVersionTime > maxVersionTime) { + maxVersionTime = visibleVersionTime; + maxPartition = hivePartition; + } + } + return new MTMVMaxTimestampSnapshot(maxPartition.getPartitionName( + hmsTable.getPartitionColumns()), maxVersionTime); + } + + private Long getPartitionIdByNameOrAnalysisException(String partitionName, + HiveMetaStoreCache.HivePartitionValues hivePartitionValues) + throws AnalysisException { + Long partitionId = hivePartitionValues.getPartitionNameToIdMap().get(partitionName); + if (partitionId == null) { + throw new AnalysisException("can not find partition: " + partitionName); + } + return partitionId; + } + + private HivePartition getHivePartitionByIdOrAnalysisException(Long partitionId, + HiveMetaStoreCache.HivePartitionValues hivePartitionValues, + HiveMetaStoreCache cache) throws AnalysisException { + List partitionValues = hivePartitionValues.getPartitionValuesMap().get(partitionId); + if (CollectionUtils.isEmpty(partitionValues)) { + throw new AnalysisException("can not find partitionValues: " + partitionId); + } + HivePartition partition = cache.getHivePartition(hmsTable.getDbName(), hmsTable.getName(), partitionValues); + if (partition == null) { + throw new AnalysisException("can not find partition: " + partitionId); + } + return partition; + } + + @Override + public boolean isPartitionColumnAllowNull() { + return true; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java index aed35dc47adef2..5b810d57a42e19 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java @@ -44,7 +44,6 @@ import com.google.common.base.Strings; import com.google.common.collect.Maps; -import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; @@ -58,9 +57,11 @@ import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -802,10 +803,10 @@ public static String showCreateTable(org.apache.hadoop.hive.metastore.api.Table return output.toString(); } - public static Schema getHudiTableSchema(HMSExternalTable table) { + public static InternalSchema getHudiTableSchema(HMSExternalTable table, boolean[] enableSchemaEvolution, + String timestamp) { HoodieTableMetaClient metaClient = table.getHudiClient(); TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient); - Schema hudiSchema; // Here, the timestamp should be reloaded again. // Because when hudi obtains the schema in `getTableAvroSchema`, it needs to read the specified commit file, @@ -815,12 +816,21 @@ public static Schema getHudiTableSchema(HMSExternalTable table) { // So, we should reload timeline so that we can read the latest commit files. metaClient.reloadActiveTimeline(); - try { - hudiSchema = HoodieAvroUtils.createHoodieWriteSchema(schemaUtil.getTableAvroSchema()); - } catch (Exception e) { - throw new RuntimeException("Cannot get hudi table schema.", e); + Option internalSchemaOption = schemaUtil.getTableInternalSchemaFromCommitMetadata(timestamp); + + if (internalSchemaOption.isPresent()) { + enableSchemaEvolution[0] = true; + return internalSchemaOption.get(); + } else { + try { + // schema evolution is not enabled. (hoodie.schema.on.read.enable = false). + enableSchemaEvolution[0] = false; + // AvroInternalSchemaConverter.convert() will generator field id. + return AvroInternalSchemaConverter.convert(schemaUtil.getTableAvroSchema(true)); + } catch (Exception e) { + throw new RuntimeException("Cannot get hudi table schema.", e); + } } - return hudiSchema; } 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 new file mode 100644 index 00000000000000..24963ec10c51cb --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HudiDlaTable.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.hive; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.PartitionItem; +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; +import org.apache.doris.datasource.hudi.HudiSchemaCacheKey; +import org.apache.doris.datasource.hudi.HudiUtils; +import org.apache.doris.datasource.mvcc.MvccSnapshot; +import org.apache.doris.mtmv.MTMVRefreshContext; +import org.apache.doris.mtmv.MTMVSnapshotIf; +import org.apache.doris.mtmv.MTMVTimestampSnapshot; + +import com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +public class HudiDlaTable extends HMSDlaTable { + + public HudiDlaTable(HMSExternalTable table) { + super(table); + } + + @Override + public PartitionType getPartitionType(Optional snapshot) { + return getPartitionColumns(snapshot).size() > 0 ? PartitionType.LIST : PartitionType.UNPARTITIONED; + } + + @Override + public Set getPartitionColumnNames(Optional snapshot) { + return getPartitionColumns(snapshot).stream() + .map(c -> c.getName().toLowerCase()).collect(Collectors.toSet()); + } + + @Override + public List getPartitionColumns(Optional snapshot) { + return getHudiSchemaCacheValue(snapshot).getPartitionColumns(); + } + + @Override + public Map getAndCopyPartitionItems(Optional snapshot) { + TablePartitionValues tablePartitionValues = getOrFetchHudiSnapshotCacheValue(snapshot); + Map idToPartitionItem = tablePartitionValues.getIdToPartitionItem(); + Map partitionIdToNameMap = tablePartitionValues.getPartitionIdToNameMap(); + Map copiedPartitionItems = Maps.newHashMap(); + for (Long key : partitionIdToNameMap.keySet()) { + copiedPartitionItems.put(partitionIdToNameMap.get(key), idToPartitionItem.get(key)); + } + return copiedPartitionItems; + } + + @Override + public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, + Optional snapshot) throws AnalysisException { + // Map partitionNameToLastModifiedMap = getOrFetchHudiSnapshotCacheValue( + // snapshot).getPartitionNameToLastModifiedMap(); + // return new MTMVTimestampSnapshot(partitionNameToLastModifiedMap.get(partitionName)); + return new MTMVTimestampSnapshot(0L); + } + + @Override + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) + throws AnalysisException { + // return new MTMVTimestampSnapshot(getOrFetchHudiSnapshotCacheValue(snapshot).getLastUpdateTimestamp()); + return new MTMVTimestampSnapshot(0L); + } + + @Override + public boolean isPartitionColumnAllowNull() { + return true; + } + + public HMSSchemaCacheValue getHudiSchemaCacheValue(Optional snapshot) { + TablePartitionValues snapshotCacheValue = getOrFetchHudiSnapshotCacheValue(snapshot); + return getHudiSchemaCacheValue(snapshotCacheValue.getLastUpdateTimestamp()); + } + + private TablePartitionValues getOrFetchHudiSnapshotCacheValue(Optional snapshot) { + if (snapshot.isPresent()) { + return ((HudiMvccSnapshot) snapshot.get()).getTablePartitionValues(); + } else { + return HudiUtils.getPartitionValues(Optional.empty(), hmsTable); + } + } + + private HMSSchemaCacheValue getHudiSchemaCacheValue(long timestamp) { + ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(hmsTable.getCatalog()); + Optional schemaCacheValue = cache.getSchemaValue( + new HudiSchemaCacheKey(hmsTable.getDbName(), hmsTable.getName(), timestamp)); + if (!schemaCacheValue.isPresent()) { + throw new CacheException("failed to getSchema for: %s.%s.%s.%s", + null, hmsTable.getCatalog().getName(), hmsTable.getDbName(), hmsTable.getName(), timestamp); + } + return (HMSSchemaCacheValue) schemaCacheValue.get(); + } +} 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 91002842b440f8..8e02158eece80f 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 @@ -42,6 +42,7 @@ import org.apache.doris.datasource.hive.HiveProperties; import org.apache.doris.datasource.hive.HiveTransaction; import org.apache.doris.datasource.hive.source.HiveSplit.HiveSplitCreator; +import org.apache.doris.datasource.mvcc.MvccUtil; import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan.SelectedPartitions; import org.apache.doris.planner.PlanNodeId; import org.apache.doris.qe.ConnectContext; @@ -128,7 +129,7 @@ protected List getPartitions() throws AnalysisException { List resPartitions = Lists.newArrayList(); HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() .getMetaStoreCache((HMSExternalCatalog) hmsTable.getCatalog()); - List partitionColumnTypes = hmsTable.getPartitionColumnTypes(); + List partitionColumnTypes = hmsTable.getPartitionColumnTypes(MvccUtil.getSnapshotFromContext(hmsTable)); if (!partitionColumnTypes.isEmpty()) { // partitioned table Collection partitionItems; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiMvccSnapshot.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiMvccSnapshot.java new file mode 100644 index 00000000000000..0f01291e54c374 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiMvccSnapshot.java @@ -0,0 +1,74 @@ +// 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.TablePartitionValues; +import org.apache.doris.datasource.mvcc.MvccSnapshot; + +/** + * Implementation of MvccSnapshot for Hudi tables that maintains partition values + * for MVCC (Multiversion Concurrency Control) operations. + * This class is immutable to ensure thread safety. + */ +public class HudiMvccSnapshot implements MvccSnapshot { + private final TablePartitionValues tablePartitionValues; + + /** + * Creates a new HudiMvccSnapshot with the specified partition values. + * + * @param tablePartitionValues The partition values for the snapshot + * @throws IllegalArgumentException if tablePartitionValues is null + */ + public HudiMvccSnapshot(TablePartitionValues tablePartitionValues) { + if (tablePartitionValues == null) { + throw new IllegalArgumentException("TablePartitionValues cannot be null"); + } + this.tablePartitionValues = tablePartitionValues; + } + + /** + * Gets the table partition values associated with this snapshot. + * + * @return The immutable TablePartitionValues object + */ + public TablePartitionValues getTablePartitionValues() { + return tablePartitionValues; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + HudiMvccSnapshot that = (HudiMvccSnapshot) o; + return tablePartitionValues.equals(that.tablePartitionValues); + } + + @Override + public int hashCode() { + return tablePartitionValues.hashCode(); + } + + @Override + public String toString() { + return String.format("HudiMvccSnapshot{tablePartitionValues=%s}", tablePartitionValues); + } +} 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 new file mode 100644 index 00000000000000..5a5b0dc044eb81 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiSchemaCacheKey.java @@ -0,0 +1,82 @@ +// 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.ExternalSchemaCache.SchemaCacheKey; + +import com.google.common.base.Objects; + +/** + * Cache key for Hudi table schemas that includes timestamp information. + * This allows for time-travel queries and ensures proper schema versioning. + */ +public class HudiSchemaCacheKey extends SchemaCacheKey { + private final long timestamp; + + /** + * Creates a new cache key for Hudi table schemas. + * + * @param dbName The database name + * @param tableName The table name + * @param timestamp The timestamp for schema version + * @throws IllegalArgumentException if dbName or tableName is null or empty + */ + public HudiSchemaCacheKey(String dbName, String tableName, long timestamp) { + super(dbName, tableName); + if (timestamp < 0) { + throw new IllegalArgumentException("Timestamp cannot be negative"); + } + this.timestamp = timestamp; + } + + /** + * Gets the timestamp associated with this schema version. + * + * @return the timestamp value + */ + public long getTimestamp() { + return timestamp; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + + HudiSchemaCacheKey that = (HudiSchemaCacheKey) o; + return timestamp == that.timestamp; + } + + @Override + public int hashCode() { + return Objects.hashCode(super.hashCode(), timestamp); + } + + @Override + public String toString() { + return String.format("HudiSchemaCacheKey{dbName='%s', tableName='%s', timestamp=%d}", + getDbName(), getTblName(), timestamp); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiSchemaCacheValue.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiSchemaCacheValue.java index 8c58ffa2006f16..5eef275fe475ce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiSchemaCacheValue.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiSchemaCacheValue.java @@ -20,14 +20,20 @@ import org.apache.doris.catalog.Column; import org.apache.doris.datasource.hive.HMSSchemaCacheValue; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.InternalSchemaCache; +import org.apache.hudi.internal.schema.InternalSchema; + import java.util.List; public class HudiSchemaCacheValue extends HMSSchemaCacheValue { private List colTypes; + boolean enableSchemaEvolution; - public HudiSchemaCacheValue(List schema, List partitionColumns) { + public HudiSchemaCacheValue(List schema, List partitionColumns, boolean enableSchemaEvolution) { super(schema, partitionColumns); + this.enableSchemaEvolution = enableSchemaEvolution; } public List getColTypes() { @@ -37,4 +43,13 @@ public List getColTypes() { public void setColTypes(List colTypes) { this.colTypes = colTypes; } + + public InternalSchema getCommitInstantInternalSchema(HoodieTableMetaClient metaClient, Long commitInstantTime) { + return InternalSchemaCache.searchSchemaAndCache(commitInstantTime, metaClient, true); + } + + public boolean isEnableSchemaEvolution() { + return enableSchemaEvolution; + } + } 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 0e8d737937ab1c..376aaaa79fe572 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 @@ -25,6 +25,9 @@ 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; @@ -42,13 +45,17 @@ import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.Types; import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; import java.text.ParseException; import java.time.LocalDate; import java.time.format.DateTimeFormatter; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; @@ -251,10 +258,6 @@ private static Type handleUnionType(Schema avroSchema) { public static TablePartitionValues getPartitionValues(Optional tableSnapshot, HMSExternalTable hmsTable) { TablePartitionValues partitionValues = new TablePartitionValues(); - if (hmsTable.getPartitionColumns().isEmpty()) { - //isn't partition table. - return partitionValues; - } HoodieTableMetaClient hudiClient = hmsTable.getHudiClient(); HudiCachedPartitionProcessor processor = (HudiCachedPartitionProcessor) Env.getCurrentEnv() @@ -296,4 +299,21 @@ public static HoodieTableMetaClient buildHudiTableMetaClient(String hudiBasePath () -> HoodieTableMetaClient.builder() .setConf(hadoopStorageConfiguration).setBasePath(hudiBasePath).build()); } + + public static Map getSchemaInfo(InternalSchema internalSchema) { + Types.RecordType record = internalSchema.getRecord(); + Map schemaInfo = new HashMap<>(record.fields().size()); + for (Types.Field field : record.fields()) { + schemaInfo.put(field.fieldId(), field.name().toLowerCase()); + } + return schemaInfo; + } + + public static HudiSchemaCacheValue getSchemaCacheValue(HMSExternalTable hmsTable, String queryInstant) { + ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(hmsTable.getCatalog()); + SchemaCacheKey key = new HudiSchemaCacheKey(hmsTable.getDbName(), hmsTable.getName(), + Long.parseLong(queryInstant)); + Optional schemaCacheValue = cache.getSchemaValue(key); + return (HudiSchemaCacheValue) schemaCacheValue.get(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedPartitionProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedPartitionProcessor.java index 62094b21c2ba1d..ef921cbfa47a3a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedPartitionProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedPartitionProcessor.java @@ -38,6 +38,7 @@ import org.slf4j.LoggerFactory; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.OptionalLong; @@ -85,14 +86,15 @@ public void cleanTablePartitions(String dbName, String tblName) { 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()) { - return null; + if (!partitionColumns.isPresent() || partitionColumns.get().length == 0) { + return partitionValues; } HoodieTimeline timeline = tableMetaClient.getCommitsAndCompactionTimeline().filterCompletedInstants(); Option lastInstant = timeline.lastInstant(); if (!lastInstant.isPresent()) { - return null; + return partitionValues; } long lastTimestamp = Long.parseLong(lastInstant.get().getTimestamp()); if (Long.parseLong(timestamp) == lastTimestamp) { @@ -100,10 +102,13 @@ public TablePartitionValues getSnapshotPartitionValues(HMSExternalTable table, } List partitionNameAndValues = getPartitionNamesBeforeOrEquals(timeline, timestamp); List partitionNames = Arrays.asList(partitionColumns.get()); - TablePartitionValues partitionValues = new TablePartitionValues(); + // 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.getPartitionColumnTypes()); + .collect(Collectors.toList()), table.getHudiPartitionColumnTypes(Long.parseLong(timestamp)), + Collections.nCopies(partitionNameAndValues.size(), 0L)); + partitionValues.setLastUpdateTimestamp(Long.parseLong(timestamp)); return partitionValues; } @@ -111,19 +116,21 @@ public TablePartitionValues getPartitionValues(HMSExternalTable table, HoodieTab boolean useHiveSyncPartition) throws CacheException { Preconditions.checkState(catalogId == table.getCatalog().getId()); + TablePartitionValues partitionValues = new TablePartitionValues(); Option partitionColumns = tableMetaClient.getTableConfig().getPartitionFields(); - if (!partitionColumns.isPresent()) { - return null; + if (!partitionColumns.isPresent() || partitionColumns.get().length == 0) { + return partitionValues; } HoodieTimeline timeline = tableMetaClient.getCommitsAndCompactionTimeline().filterCompletedInstants(); Option lastInstant = timeline.lastInstant(); if (!lastInstant.isPresent()) { - return null; + return partitionValues; } try { long lastTimestamp = Long.parseLong(lastInstant.get().getTimestamp()); - TablePartitionValues partitionValues = partitionCache.get( - new TablePartitionKey(table.getDbName(), table.getName(), table.getPartitionColumnTypes())); + partitionValues = partitionCache.get( + new TablePartitionKey(table.getDbName(), table.getName(), + table.getHudiPartitionColumnTypes(lastTimestamp))); partitionValues.readLock().lock(); try { long lastUpdateTimestamp = partitionValues.getLastUpdateTimestamp(); @@ -159,7 +166,8 @@ public TablePartitionValues getPartitionValues(HMSExternalTable table, HoodieTab partitionValues.cleanPartitions(); partitionValues.addPartitions(partitionNames, partitionNames.stream().map(p -> parsePartitionValues(partitionColumnsList, p)) - .collect(Collectors.toList()), table.getPartitionColumnTypes()); + .collect(Collectors.toList()), table.getHudiPartitionColumnTypes(lastTimestamp), + Collections.nCopies(partitionNames.size(), 0L)); partitionValues.setLastUpdateTimestamp(lastTimestamp); return partitionValues; } finally { 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 f14e02f3e587b5..987adbdc0bc018 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 @@ -29,14 +29,14 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.util.FileFormatUtils; import org.apache.doris.common.util.LocationPath; -import org.apache.doris.datasource.ExternalSchemaCache; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.FileSplit; -import org.apache.doris.datasource.SchemaCacheValue; 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.HudiSchemaCacheValue; +import org.apache.doris.datasource.hudi.HudiUtils; +import org.apache.doris.datasource.mvcc.MvccUtil; import org.apache.doris.planner.PlanNodeId; import org.apache.doris.qe.SessionVariable; import org.apache.doris.spi.Split; @@ -163,11 +163,6 @@ protected void doInitialize() throws UserException { basePath = hmsTable.getRemoteTable().getSd().getLocation(); inputFormat = hmsTable.getRemoteTable().getSd().getInputFormat(); serdeLib = hmsTable.getRemoteTable().getSd().getSerdeInfo().getSerializationLib(); - ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(table.getCatalog()); - Optional schemaCacheValue = cache.getSchemaValue(table.getDbName(), table.getName()); - HudiSchemaCacheValue hudiSchemaCacheValue = (HudiSchemaCacheValue) schemaCacheValue.get(); - columnNames = hudiSchemaCacheValue.getSchema().stream().map(Column::getName).collect(Collectors.toList()); - columnTypes = hudiSchemaCacheValue.getColTypes(); if (scanParams != null && !scanParams.incrementalRead()) { // Only support incremental read @@ -208,6 +203,11 @@ protected void doInitialize() throws UserException { } queryInstant = snapshotInstant.get().getTimestamp(); } + + HudiSchemaCacheValue hudiSchemaCacheValue = HudiUtils.getSchemaCacheValue(hmsTable, queryInstant); + columnNames = hudiSchemaCacheValue.getSchema().stream().map(Column::getName).collect(Collectors.toList()); + columnTypes = hudiSchemaCacheValue.getColTypes(); + fsView = Env.getCurrentEnv() .getExtMetaCacheMgr() .getFsViewProcessor(hmsTable.getCatalog()) @@ -271,7 +271,7 @@ private boolean canUseNativeReader() { } private List getPrunedPartitions(HoodieTableMetaClient metaClient) { - List partitionColumnTypes = hmsTable.getPartitionColumnTypes(); + List partitionColumnTypes = hmsTable.getPartitionColumnTypes(MvccUtil.getSnapshotFromContext(hmsTable)); if (!partitionColumnTypes.isEmpty()) { this.totalPartitionNum = selectedPartitions.totalPartitionNum; Map prunedPartitions = selectedPartitions.selectedPartitions; 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 472b792f6d1923..e9bffbf56e3623 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 @@ -17,11 +17,32 @@ package org.apache.doris.datasource.iceberg; +import org.apache.doris.analysis.PartitionValue; +import org.apache.doris.analysis.TableSnapshot; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.catalog.PartitionKey; +import org.apache.doris.catalog.PartitionType; +import org.apache.doris.catalog.RangePartitionItem; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; +import org.apache.doris.datasource.CacheException; +import org.apache.doris.datasource.ExternalSchemaCache; +import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.SchemaCacheValue; +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.SupportedSysTables; import org.apache.doris.datasource.systable.SysTable; +import org.apache.doris.mtmv.MTMVBaseTableIf; +import org.apache.doris.mtmv.MTMVRefreshContext; +import org.apache.doris.mtmv.MTMVRelatedTableIf; +import org.apache.doris.mtmv.MTMVSnapshotIdSnapshot; +import org.apache.doris.mtmv.MTMVSnapshotIf; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ExternalAnalysisTask; @@ -30,13 +51,53 @@ import org.apache.doris.thrift.TTableDescriptor; import org.apache.doris.thrift.TTableType; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Range; +import com.google.common.collect.Sets; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionsTable; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.StructProjection; +import java.io.IOException; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.Month; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Comparator; import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; -public class IcebergExternalTable extends ExternalTable { +public class IcebergExternalTable extends ExternalTable implements MTMVRelatedTableIf, MTMVBaseTableIf, MvccTable { + + public static final String YEAR = "year"; + public static final String MONTH = "month"; + public static final String DAY = "day"; + public static final String HOUR = "hour"; + public static final String IDENTITY = "identity"; + public static final int PARTITION_DATA_ID_START = 1000; // org.apache.iceberg.PartitionSpec + + private Table table; + private List partitionColumns; + private boolean isValidRelatedTableCached = false; + private boolean isValidRelatedTable = false; public IcebergExternalTable(long id, String name, String remoteName, IcebergExternalCatalog catalog, IcebergExternalDatabase db) { @@ -54,9 +115,34 @@ protected synchronized void makeSureInitialized() { } } + @VisibleForTesting + public void setTable(Table table) { + this.table = table; + } + + @VisibleForTesting + public void setPartitionColumns(List partitionColumns) { + this.partitionColumns = partitionColumns; + } + @Override - public Optional initSchema() { - return Optional.of(new SchemaCacheValue(IcebergUtils.getSchema(catalog, dbName, name))); + public Optional initSchema(SchemaCacheKey key) { + table = getIcebergTable(); + List schema = IcebergUtils.getSchema(catalog, dbName, name, + ((IcebergSchemaCacheKey) key).getSchemaId()); + List tmpColumns = Lists.newArrayList(); + PartitionSpec spec = table.spec(); + for (PartitionField field : spec.fields()) { + Types.NestedField col = table.schema().findField(field.sourceId()); + for (Column c : schema) { + if (c.getName().equalsIgnoreCase(col.name())) { + tmpColumns.add(c); + break; + } + } + } + partitionColumns = tmpColumns; + return Optional.of(new IcebergSchemaCacheValue(schema, partitionColumns)); } @Override @@ -94,6 +180,391 @@ public Table getIcebergTable() { return IcebergUtils.getIcebergTable(getCatalog(), getDbName(), getName()); } + private IcebergSnapshotCacheValue getIcebergSnapshotCacheValue() { + return Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache() + .getSnapshotCache(catalog, dbName, name); + } + + @Override + public void beforeMTMVRefresh(MTMV mtmv) throws DdlException { + } + + @Override + public Map getAndCopyPartitionItems(Optional snapshot) { + return Maps.newHashMap(getOrFetchSnapshotCacheValue(snapshot).getPartitionInfo().getNameToPartitionItem()); + } + + @Override + public Map getNameToPartitionItems(Optional snapshot) { + return getOrFetchSnapshotCacheValue(snapshot).getPartitionInfo().getNameToPartitionItem(); + } + + @Override + public PartitionType getPartitionType(Optional snapshot) { + return isValidRelatedTable() ? PartitionType.RANGE : PartitionType.UNPARTITIONED; + } + + @Override + public Set getPartitionColumnNames(Optional snapshot) throws DdlException { + return getPartitionColumns(snapshot).stream().map(Column::getName).collect(Collectors.toSet()); + } + + @Override + public List getPartitionColumns(Optional snapshot) { + IcebergSnapshotCacheValue snapshotValue = getOrFetchSnapshotCacheValue(snapshot); + IcebergSchemaCacheValue schemaValue = getIcebergSchemaCacheValue(snapshotValue.getSnapshot().getSchemaId()); + return schemaValue.getPartitionColumns(); + } + + @Override + public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, + Optional snapshot) throws AnalysisException { + IcebergSnapshotCacheValue snapshotValue = getOrFetchSnapshotCacheValue(snapshot); + long latestSnapshotId = snapshotValue.getPartitionInfo().getLatestSnapshotId(partitionName); + if (latestSnapshotId <= 0) { + throw new AnalysisException("can not find partition: " + partitionName); + } + return new MTMVSnapshotIdSnapshot(latestSnapshotId); + } + + @Override + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) + throws AnalysisException { + makeSureInitialized(); + IcebergSnapshotCacheValue snapshotValue = getOrFetchSnapshotCacheValue(snapshot); + return new MTMVSnapshotIdSnapshot(snapshotValue.getSnapshot().getSnapshotId()); + } + + @Override + public boolean isPartitionColumnAllowNull() { + return true; + } + + /** + * For now, we only support single partition column Iceberg table as related table. + * The supported transforms now are YEAR, MONTH, DAY and HOUR. + * And the column couldn't change to another column during partition evolution. + */ + @Override + public boolean isValidRelatedTable() { + makeSureInitialized(); + if (isValidRelatedTableCached) { + return isValidRelatedTable; + } + isValidRelatedTable = false; + Set allFields = Sets.newHashSet(); + table = getIcebergTable(); + for (PartitionSpec spec : table.specs().values()) { + if (spec == null) { + isValidRelatedTableCached = true; + return false; + } + List fields = spec.fields(); + if (fields.size() != 1) { + isValidRelatedTableCached = true; + return false; + } + PartitionField partitionField = spec.fields().get(0); + String transformName = partitionField.transform().toString(); + if (!YEAR.equals(transformName) + && !MONTH.equals(transformName) + && !DAY.equals(transformName) + && !HOUR.equals(transformName)) { + isValidRelatedTableCached = true; + return false; + } + allFields.add(table.schema().findColumnName(partitionField.sourceId())); + } + isValidRelatedTableCached = true; + isValidRelatedTable = allFields.size() == 1; + return isValidRelatedTable; + } + + @Override + public MvccSnapshot loadSnapshot(Optional tableSnapshot) { + return new IcebergMvccSnapshot(getIcebergSnapshotCacheValue()); + } + + public long getLatestSnapshotId() { + table = getIcebergTable(); + Snapshot snapshot = table.currentSnapshot(); + return snapshot == null ? IcebergUtils.UNKNOWN_SNAPSHOT_ID : table.currentSnapshot().snapshotId(); + } + + public long getSchemaId(long snapshotId) { + table = getIcebergTable(); + return snapshotId == IcebergUtils.UNKNOWN_SNAPSHOT_ID + ? IcebergUtils.UNKNOWN_SNAPSHOT_ID + : table.snapshot(snapshotId).schemaId(); + } + + @Override + public List getFullSchema() { + Optional snapshotFromContext = MvccUtil.getSnapshotFromContext(this); + IcebergSnapshotCacheValue cacheValue = getOrFetchSnapshotCacheValue(snapshotFromContext); + return getIcebergSchemaCacheValue(cacheValue.getSnapshot().getSchemaId()).getSchema(); + } + + @Override + public boolean supportInternalPartitionPruned() { + return true; + } + + public IcebergSchemaCacheValue getIcebergSchemaCacheValue(long schemaId) { + ExternalSchemaCache cache = Env.getCurrentEnv().getExtMetaCacheMgr().getSchemaCache(catalog); + Optional schemaCacheValue = cache.getSchemaValue( + new IcebergSchemaCacheKey(dbName, name, schemaId)); + if (!schemaCacheValue.isPresent()) { + throw new CacheException("failed to getSchema for: %s.%s.%s.%s", + null, catalog.getName(), dbName, name, schemaId); + } + return (IcebergSchemaCacheValue) schemaCacheValue.get(); + } + + public IcebergPartitionInfo loadPartitionInfo(long snapshotId) throws AnalysisException { + // snapshotId == UNKNOWN_SNAPSHOT_ID means this is an empty table, haven't contained any snapshot yet. + if (!isValidRelatedTable() || snapshotId == IcebergUtils.UNKNOWN_SNAPSHOT_ID) { + return new IcebergPartitionInfo(); + } + List icebergPartitions = loadIcebergPartition(snapshotId); + Map nameToPartition = Maps.newHashMap(); + Map nameToPartitionItem = Maps.newHashMap(); + table = getIcebergTable(); + partitionColumns = getIcebergSchemaCacheValue(table.snapshot(snapshotId).schemaId()).getPartitionColumns(); + for (IcebergPartition partition : icebergPartitions) { + nameToPartition.put(partition.getPartitionName(), partition); + String transform = table.specs().get(partition.getSpecId()).fields().get(0).transform().toString(); + Range partitionRange = getPartitionRange( + partition.getPartitionValues().get(0), transform, partitionColumns); + PartitionItem item = new RangePartitionItem(partitionRange); + nameToPartitionItem.put(partition.getPartitionName(), item); + } + Map> partitionNameMap = mergeOverlapPartitions(nameToPartitionItem); + return new IcebergPartitionInfo(nameToPartitionItem, nameToPartition, partitionNameMap); + } + + public List loadIcebergPartition(long snapshotId) { + PartitionsTable partitionsTable = (PartitionsTable) MetadataTableUtils + .createMetadataTableInstance(table, MetadataTableType.PARTITIONS); + List partitions = Lists.newArrayList(); + try (CloseableIterable tasks = partitionsTable.newScan().useSnapshot(snapshotId).planFiles()) { + for (FileScanTask task : tasks) { + CloseableIterable rows = task.asDataTask().rows(); + for (StructLike row : rows) { + partitions.add(generateIcebergPartition(row)); + } + } + } catch (IOException e) { + LOG.warn("Failed to get Iceberg table {} partition info.", name, e); + } + return partitions; + } + + public IcebergPartition generateIcebergPartition(StructLike row) { + // row format : + // 0. partitionData, + // 1. spec_id, + // 2. record_count, + // 3. file_count, + // 4. total_data_file_size_in_bytes, + // 5. position_delete_record_count, + // 6. position_delete_file_count, + // 7. equality_delete_record_count, + // 8. equality_delete_file_count, + // 9. last_updated_at, + // 10. last_updated_snapshot_id + table = getIcebergTable(); + Preconditions.checkState(!table.spec().fields().isEmpty(), table.name() + " is not a partition table."); + int specId = row.get(1, Integer.class); + PartitionSpec partitionSpec = table.specs().get(specId); + StructProjection partitionData = row.get(0, StructProjection.class); + StringBuilder sb = new StringBuilder(); + List partitionValues = Lists.newArrayList(); + List transforms = Lists.newArrayList(); + for (int i = 0; i < partitionSpec.fields().size(); ++i) { + PartitionField partitionField = partitionSpec.fields().get(i); + Class fieldClass = partitionSpec.javaClasses()[i]; + int fieldId = partitionField.fieldId(); + // Iceberg partition field id starts at PARTITION_DATA_ID_START, + // So we can get the field index in partitionData using fieldId - PARTITION_DATA_ID_START + int index = fieldId - PARTITION_DATA_ID_START; + Object o = partitionData.get(index, fieldClass); + String fieldValue = o == null ? null : o.toString(); + String fieldName = partitionField.name(); + sb.append(fieldName); + sb.append("="); + sb.append(fieldValue); + sb.append("/"); + partitionValues.add(fieldValue); + transforms.add(partitionField.transform().toString()); + } + if (sb.length() > 0) { + sb.delete(sb.length() - 1, sb.length()); + } + String partitionName = sb.toString(); + long recordCount = row.get(2, Long.class); + long fileCount = row.get(3, Integer.class); + long fileSizeInBytes = row.get(4, Long.class); + long lastUpdateTime = row.get(9, Long.class); + long lastUpdateSnapShotId = row.get(10, Long.class); + return new IcebergPartition(partitionName, specId, recordCount, fileSizeInBytes, fileCount, + lastUpdateTime, lastUpdateSnapShotId, partitionValues, transforms); + } + + @VisibleForTesting + public Range getPartitionRange(String value, String transform, List partitionColumns) + throws AnalysisException { + // For NULL value, create a minimum partition for it. + if (value == null) { + PartitionKey nullLowKey = PartitionKey.createPartitionKey( + Lists.newArrayList(new PartitionValue("0000-01-01")), partitionColumns); + PartitionKey nullUpKey = nullLowKey.successor(); + return Range.closedOpen(nullLowKey, nullUpKey); + } + LocalDateTime epoch = Instant.EPOCH.atZone(ZoneId.of("UTC")).toLocalDateTime(); + LocalDateTime target; + LocalDateTime lower; + LocalDateTime upper; + long longValue = Long.parseLong(value); + switch (transform) { + case HOUR: + target = epoch.plusHours(longValue); + lower = LocalDateTime.of(target.getYear(), target.getMonth(), target.getDayOfMonth(), + target.getHour(), 0, 0); + upper = lower.plusHours(1); + break; + case DAY: + target = epoch.plusDays(longValue); + lower = LocalDateTime.of(target.getYear(), target.getMonth(), target.getDayOfMonth(), 0, 0, 0); + upper = lower.plusDays(1); + break; + case MONTH: + target = epoch.plusMonths(longValue); + lower = LocalDateTime.of(target.getYear(), target.getMonth(), 1, 0, 0, 0); + upper = lower.plusMonths(1); + break; + case YEAR: + target = epoch.plusYears(longValue); + lower = LocalDateTime.of(target.getYear(), Month.JANUARY, 1, 0, 0, 0); + upper = lower.plusYears(1); + break; + default: + throw new RuntimeException("Unsupported transform " + transform); + } + DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); + Column c = partitionColumns.get(0); + Preconditions.checkState(c.getDataType().isDateType(), "Only support date type partition column"); + if (c.getType().isDate() || c.getType().isDateV2()) { + formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd"); + } + PartitionValue lowerValue = new PartitionValue(lower.format(formatter)); + PartitionValue upperValue = new PartitionValue(upper.format(formatter)); + PartitionKey lowKey = PartitionKey.createPartitionKey(Lists.newArrayList(lowerValue), partitionColumns); + PartitionKey upperKey = PartitionKey.createPartitionKey(Lists.newArrayList(upperValue), partitionColumns); + return Range.closedOpen(lowKey, upperKey); + } + + /** + * Merge overlapped iceberg partitions into one Doris partition. + */ + public Map> mergeOverlapPartitions(Map originPartitions) { + List> entries = sortPartitionMap(originPartitions); + Map> map = Maps.newHashMap(); + for (int i = 0; i < entries.size() - 1; i++) { + Range firstValue = entries.get(i).getValue().getItems(); + String firstKey = entries.get(i).getKey(); + Range secondValue = entries.get(i + 1).getValue().getItems(); + String secondKey = entries.get(i + 1).getKey(); + // If the first entry enclose the second one, remove the second entry and keep a record in the return map. + // So we can track the iceberg partitions those contained by one Doris partition. + while (i < entries.size() && firstValue.encloses(secondValue)) { + originPartitions.remove(secondKey); + map.putIfAbsent(firstKey, Sets.newHashSet(firstKey)); + String finalSecondKey = secondKey; + map.computeIfPresent(firstKey, (key, value) -> { + value.add(finalSecondKey); + return value; + }); + i++; + if (i >= entries.size() - 1) { + break; + } + secondValue = entries.get(i + 1).getValue().getItems(); + secondKey = entries.get(i + 1).getKey(); + } + } + return map; + } + + /** + * Sort the given map entries by PartitionItem Range(LOW, HIGH) + * When comparing two ranges, the one with smaller LOW value is smaller than the other one. + * If two ranges have same values of LOW, the one with larger HIGH value is smaller. + * + * For now, we only support year, month, day and hour, + * so it is impossible to have two partially intersect partitions. + * One range is either enclosed by another or has no intersection at all with another. + * + * + * For example, we have these 4 ranges: + * [10, 20), [30, 40), [0, 30), [10, 15) + * + * After sort, they become: + * [0, 30), [10, 20), [10, 15), [30, 40) + */ + public List> sortPartitionMap(Map originPartitions) { + List> entries = new ArrayList<>(originPartitions.entrySet()); + entries.sort(new RangeComparator()); + return entries; + } + + public static class RangeComparator implements Comparator> { + @Override + public int compare(Map.Entry p1, Map.Entry p2) { + PartitionItem value1 = p1.getValue(); + PartitionItem value2 = p2.getValue(); + if (value1 instanceof RangePartitionItem && value2 instanceof RangePartitionItem) { + Range items1 = value1.getItems(); + Range items2 = value2.getItems(); + if (!items1.hasLowerBound()) { + return -1; + } + if (!items2.hasLowerBound()) { + return 1; + } + PartitionKey upper1 = items1.upperEndpoint(); + PartitionKey lower1 = items1.lowerEndpoint(); + PartitionKey upper2 = items2.upperEndpoint(); + PartitionKey lower2 = items2.lowerEndpoint(); + int compareLow = lower1.compareTo(lower2); + return compareLow == 0 ? upper2.compareTo(upper1) : compareLow; + } + return 0; + } + } + + @VisibleForTesting + public boolean isValidRelatedTableCached() { + return isValidRelatedTableCached; + } + + @VisibleForTesting + public boolean validRelatedTableCache() { + return isValidRelatedTable; + } + + public void setIsValidRelatedTableCached(boolean isCached) { + this.isValidRelatedTableCached = isCached; + } + + public IcebergSnapshotCacheValue getOrFetchSnapshotCacheValue(Optional snapshot) { + if (snapshot.isPresent()) { + return ((IcebergMvccSnapshot) snapshot.get()).getSnapshotCacheValue(); + } else { + return getIcebergSnapshotCacheValue(); + } + } + @Override public List getSupportedSysTables() { makeSureInitialized(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java index e209fffa06833f..f3daf2d27950f1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java @@ -18,6 +18,7 @@ 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.common.UserException; @@ -49,6 +50,7 @@ public class IcebergMetadataCache { private final LoadingCache> snapshotListCache; private final LoadingCache tableCache; + private final LoadingCache snapshotCache; public IcebergMetadataCache(ExecutorService executor) { CacheFactory snapshotListCacheFactory = new CacheFactory( @@ -66,6 +68,14 @@ public IcebergMetadataCache(ExecutorService executor) { true, null); this.tableCache = tableCacheFactory.buildCache(key -> loadTable(key), null, executor); + + CacheFactory snapshotCacheFactory = new CacheFactory( + OptionalLong.of(28800L), + OptionalLong.of(Config.external_cache_expire_time_minutes_after_access * 60), + Config.max_external_table_cache_num, + true, + null); + this.snapshotCache = snapshotCacheFactory.buildCache(key -> loadSnapshot(key), null, executor); } public List getSnapshotList(TIcebergMetadataParams params) throws UserException { @@ -92,6 +102,11 @@ public Table getAndCloneTable(CatalogIf catalog, String dbName, String tbName) { return restTable; } + public IcebergSnapshotCacheValue getSnapshotCache(CatalogIf catalog, String dbName, String tbName) { + IcebergMetadataCacheKey key = IcebergMetadataCacheKey.of(catalog, dbName, tbName); + return snapshotCache.get(key); + } + @NotNull private List loadSnapshots(IcebergMetadataCacheKey key) { Table icebergTable = getIcebergTable(key.catalog, key.dbName, key.tableName); @@ -119,6 +134,16 @@ private Table loadTable(IcebergMetadataCacheKey key) { } + @NotNull + private IcebergSnapshotCacheValue loadSnapshot(IcebergMetadataCacheKey key) throws AnalysisException { + IcebergExternalTable table = (IcebergExternalTable) key.catalog.getDbOrAnalysisException(key.dbName) + .getTableOrAnalysisException(key.tableName); + long snapshotId = table.getLatestSnapshotId(); + long schemaId = table.getSchemaId(snapshotId); + IcebergPartitionInfo icebergPartitionInfo = table.loadPartitionInfo(snapshotId); + return new IcebergSnapshotCacheValue(icebergPartitionInfo, new IcebergSnapshot(snapshotId, schemaId)); + } + public void invalidateCatalogCache(long catalogId) { snapshotListCache.asMap().keySet().stream() .filter(key -> key.catalog.getId() == catalogId) @@ -130,6 +155,10 @@ public void invalidateCatalogCache(long catalogId) { ManifestFiles.dropCache(entry.getValue().io()); tableCache.invalidate(entry.getKey()); }); + + snapshotCache.asMap().keySet().stream() + .filter(key -> key.catalog.getId() == catalogId) + .forEach(snapshotCache::invalidate); } public void invalidateTableCache(long catalogId, String dbName, String tblName) { @@ -148,6 +177,11 @@ public void invalidateTableCache(long catalogId, String dbName, String tblName) ManifestFiles.dropCache(entry.getValue().io()); tableCache.invalidate(entry.getKey()); }); + + snapshotCache.asMap().keySet().stream() + .filter(key -> key.catalog.getId() == catalogId && key.dbName.equals(dbName) && key.tableName.equals( + tblName)) + .forEach(snapshotCache::invalidate); } public void invalidateDbCache(long catalogId, String dbName) { @@ -164,6 +198,10 @@ public void invalidateDbCache(long catalogId, String dbName) { ManifestFiles.dropCache(entry.getValue().io()); tableCache.invalidate(entry.getKey()); }); + + snapshotCache.asMap().keySet().stream() + .filter(key -> key.catalog.getId() == catalogId && key.dbName.equals(dbName)) + .forEach(snapshotCache::invalidate); } private static void initIcebergTableFileIO(Table table, Map props) { @@ -217,10 +255,12 @@ public int hashCode() { public Map> getCacheStats() { Map> res = Maps.newHashMap(); - res.put("iceberg_snapshot_cache", ExternalMetaCacheMgr.getCacheStats(snapshotListCache.stats(), + res.put("iceberg_snapshot_list_cache", ExternalMetaCacheMgr.getCacheStats(snapshotListCache.stats(), snapshotListCache.estimatedSize())); res.put("iceberg_table_cache", ExternalMetaCacheMgr.getCacheStats(tableCache.stats(), tableCache.estimatedSize())); + res.put("iceberg_snapshot_cache", ExternalMetaCacheMgr.getCacheStats(snapshotCache.stats(), + snapshotCache.estimatedSize())); return res; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMvccSnapshot.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMvccSnapshot.java new file mode 100644 index 00000000000000..2c0155a71cd389 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMvccSnapshot.java @@ -0,0 +1,32 @@ +// 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.mvcc.MvccSnapshot; + +public class IcebergMvccSnapshot implements MvccSnapshot { + private final IcebergSnapshotCacheValue snapshotCacheValue; + + public IcebergMvccSnapshot(IcebergSnapshotCacheValue snapshotCacheValue) { + this.snapshotCacheValue = snapshotCacheValue; + } + + public IcebergSnapshotCacheValue getSnapshotCacheValue() { + return snapshotCacheValue; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergPartition.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergPartition.java new file mode 100644 index 00000000000000..cccc6244a0d0cc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergPartition.java @@ -0,0 +1,82 @@ +// 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 java.util.List; + +public class IcebergPartition { + private final String partitionName; + private final List partitionValues; + private final int specId; + private final long recordCount; + private final long fileSizeInBytes; + private final long fileCount; + private final long lastUpdateTime; + private final long lastSnapshotId; + private final List transforms; + + public IcebergPartition(String partitionName, int specId, long recordCount, long fileSizeInBytes, long fileCount, + long lastUpdateTime, long lastSnapshotId, List partitionValues, + List transforms) { + this.partitionName = partitionName; + this.specId = specId; + this.recordCount = recordCount; + this.fileSizeInBytes = fileSizeInBytes; + this.fileCount = fileCount; + this.lastUpdateTime = lastUpdateTime; + this.lastSnapshotId = lastSnapshotId; + this.partitionValues = partitionValues; + this.transforms = transforms; + } + + public String getPartitionName() { + return partitionName; + } + + public int getSpecId() { + return specId; + } + + public long getRecordCount() { + return recordCount; + } + + public long getFileSizeInBytes() { + return fileSizeInBytes; + } + + public long getFileCount() { + return fileCount; + } + + public long getLastUpdateTime() { + return lastUpdateTime; + } + + public long getLastSnapshotId() { + return lastSnapshotId; + } + + public List getPartitionValues() { + return partitionValues; + } + + public List getTransforms() { + return transforms; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergPartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergPartitionInfo.java new file mode 100644 index 00000000000000..9edb2137f4f389 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergPartitionInfo.java @@ -0,0 +1,71 @@ +// 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.PartitionItem; + +import com.google.common.collect.Maps; + +import java.util.Map; +import java.util.Set; + +public class IcebergPartitionInfo { + private final Map nameToPartitionItem; + private final Map nameToIcebergPartition; + private final Map> nameToIcebergPartitionNames; + + public IcebergPartitionInfo() { + this.nameToPartitionItem = Maps.newHashMap(); + this.nameToIcebergPartition = Maps.newHashMap(); + this.nameToIcebergPartitionNames = Maps.newHashMap(); + } + + public IcebergPartitionInfo(Map nameToPartitionItem, + Map nameToIcebergPartition, + Map> nameToIcebergPartitionNames) { + this.nameToPartitionItem = nameToPartitionItem; + this.nameToIcebergPartition = nameToIcebergPartition; + this.nameToIcebergPartitionNames = nameToIcebergPartitionNames; + } + + public Map getNameToPartitionItem() { + return nameToPartitionItem; + } + + public Map getNameToIcebergPartition() { + return nameToIcebergPartition; + } + + public long getLatestSnapshotId(String partitionName) { + Set icebergPartitionNames = nameToIcebergPartitionNames.get(partitionName); + if (icebergPartitionNames == null) { + return nameToIcebergPartition.get(partitionName).getLastSnapshotId(); + } + long latestSnapshotId = 0; + long latestUpdateTime = -1; + for (String name : icebergPartitionNames) { + IcebergPartition partition = nameToIcebergPartition.get(name); + long lastUpdateTime = partition.getLastUpdateTime(); + if (latestUpdateTime < lastUpdateTime) { + latestUpdateTime = lastUpdateTime; + latestSnapshotId = partition.getLastSnapshotId(); + } + } + return latestSnapshotId; + } +} 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 new file mode 100644 index 00000000000000..7931d91831fcec --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSchemaCacheKey.java @@ -0,0 +1,55 @@ +// 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.ExternalSchemaCache.SchemaCacheKey; + +import com.google.common.base.Objects; + +public class IcebergSchemaCacheKey extends SchemaCacheKey { + private final long schemaId; + + public IcebergSchemaCacheKey(String dbName, String tableName, long schemaId) { + super(dbName, tableName); + this.schemaId = schemaId; + } + + public long getSchemaId() { + return schemaId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof IcebergSchemaCacheKey)) { + return false; + } + if (!super.equals(o)) { + return false; + } + IcebergSchemaCacheKey that = (IcebergSchemaCacheKey) o; + return schemaId == that.schemaId; + } + + @Override + public int hashCode() { + return Objects.hashCode(super.hashCode(), schemaId); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSchemaCacheValue.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSchemaCacheValue.java new file mode 100644 index 00000000000000..ccfcaab0c7261d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSchemaCacheValue.java @@ -0,0 +1,37 @@ +// 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.Column; +import org.apache.doris.datasource.SchemaCacheValue; + +import java.util.List; + +public class IcebergSchemaCacheValue extends SchemaCacheValue { + + private final List partitionColumns; + + public IcebergSchemaCacheValue(List schema, List partitionColumns) { + super(schema); + this.partitionColumns = partitionColumns; + } + + public List getPartitionColumns() { + return partitionColumns; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSnapshot.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSnapshot.java new file mode 100644 index 00000000000000..5903c362d7434e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSnapshot.java @@ -0,0 +1,36 @@ +// 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; + +public class IcebergSnapshot { + private final long snapshotId; + private final long schemaId; + + public IcebergSnapshot(long snapshotId, long schemaId) { + this.snapshotId = snapshotId; + this.schemaId = schemaId; + } + + public long getSnapshotId() { + return snapshotId; + } + + public long getSchemaId() { + return schemaId; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSnapshotCacheValue.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSnapshotCacheValue.java new file mode 100644 index 00000000000000..95c9a6f26cc5c5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSnapshotCacheValue.java @@ -0,0 +1,37 @@ +// 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; + +public class IcebergSnapshotCacheValue { + + private final IcebergPartitionInfo partitionInfo; + private final IcebergSnapshot snapshot; + + public IcebergSnapshotCacheValue(IcebergPartitionInfo partitionInfo, IcebergSnapshot snapshot) { + this.partitionInfo = partitionInfo; + this.snapshot = snapshot; + } + + public IcebergPartitionInfo getPartitionInfo() { + return partitionInfo; + } + + public IcebergSnapshot getSnapshot() { + return snapshot; + } +} 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 75b9b9e1812a50..ab0d84a2b26343 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 @@ -47,17 +47,20 @@ import org.apache.doris.common.info.SimpleTableInfo; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.SchemaCacheValue; import org.apache.doris.datasource.property.constants.HMSProperties; import org.apache.doris.nereids.exceptions.NotSupportedException; import org.apache.doris.thrift.TExprOpcode; import com.github.benmanes.caffeine.cache.Caffeine; import com.github.benmanes.caffeine.cache.LoadingCache; +import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.FileFormat; import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; @@ -83,6 +86,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; /** @@ -113,6 +117,8 @@ public Integer initialValue() { // nickname in spark public static final String SPARK_SQL_COMPRESSION_CODEC = "spark.sql.iceberg.compression-codec"; + public static final long UNKNOWN_SNAPSHOT_ID = -1; + public static Expression convertToIcebergExpr(Expr expr, Schema schema) { if (expr == null) { return null; @@ -576,14 +582,25 @@ private static org.apache.iceberg.Table getIcebergTableInternal(ExternalCatalog : metadataCache.getIcebergTable(catalog, dbName, tblName); } + public static List getSchema(ExternalCatalog catalog, String dbName, String name) { + return getSchema(catalog, dbName, name, UNKNOWN_SNAPSHOT_ID); + } + /** * Get iceberg schema from catalog and convert them to doris schema */ - public static List getSchema(ExternalCatalog catalog, String dbName, String name) { + public static List getSchema(ExternalCatalog catalog, String dbName, String name, long schemaId) { try { return catalog.getPreExecutionAuthenticator().execute(() -> { org.apache.iceberg.Table icebergTable = getIcebergTable(catalog, dbName, name); - Schema schema = icebergTable.schema(); + Schema schema; + if (schemaId == UNKNOWN_SNAPSHOT_ID || icebergTable.currentSnapshot() == null) { + schema = icebergTable.schema(); + } else { + schema = icebergTable.schemas().get((int) schemaId); + } + Preconditions.checkNotNull(schema, + "Schema for table " + catalog.getName() + "." + dbName + "." + name + " is null"); List columns = schema.columns(); List tmpSchema = Lists.newArrayListWithCapacity(columns.size()); for (Types.NestedField field : columns) { @@ -702,6 +719,25 @@ public static HiveCatalog createIcebergHiveCatalog(ExternalCatalog externalCatal return hiveCatalog; } + // load table schema from iceberg API to external schema cache. + public static Optional loadSchemaCacheValue( + ExternalCatalog catalog, String dbName, String tbName, long schemaId) { + Table table = IcebergUtils.getIcebergTable(catalog, dbName, tbName); + List schema = IcebergUtils.getSchema(catalog, dbName, tbName, schemaId); + List tmpColumns = Lists.newArrayList(); + PartitionSpec spec = table.spec(); + for (PartitionField field : spec.fields()) { + Types.NestedField col = table.schema().findField(field.sourceId()); + for (Column c : schema) { + if (c.getName().equalsIgnoreCase(col.name())) { + tmpColumns.add(c); + break; + } + } + } + return Optional.of(new IcebergSchemaCacheValue(schema, tmpColumns)); + } + // Retrieve the manifest files that match the query based on partitions in filter public static CloseableIterable getMatchingManifest( List dataManifests, 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 1136fb079f3bcd..a598c68703fa39 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 @@ -134,7 +134,7 @@ private TablePartitionValues loadPartitionValues(MaxComputeSchemaCacheValue sche partitionSpecs.stream() .map(p -> parsePartitionValues(partitionColumnNames, p)) .collect(Collectors.toList()), - partitionTypes); + partitionTypes, Collections.nCopies(partitionSpecs.size(), 0L)); return partitionValues; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/EmptyMvccSnapshot.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/EmptyMvccSnapshot.java new file mode 100644 index 00000000000000..35f63291a258c8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/EmptyMvccSnapshot.java @@ -0,0 +1,21 @@ +// 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.mvcc; + +public class EmptyMvccSnapshot implements MvccSnapshot { +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/MvccTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/MvccTable.java index d69e0f3114df0c..89b1d6e9b07928 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/MvccTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/MvccTable.java @@ -17,8 +17,11 @@ package org.apache.doris.datasource.mvcc; +import org.apache.doris.analysis.TableSnapshot; import org.apache.doris.catalog.TableIf; +import java.util.Optional; + /** * The table that needs to query data based on the version needs to implement this interface. */ @@ -29,5 +32,5 @@ public interface MvccTable extends TableIf { * * @return MvccSnapshot */ - MvccSnapshot loadSnapshot(); + MvccSnapshot loadSnapshot(Optional tableSnapshot); } 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 99c4d802347396..a6124649e2533d 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 @@ -17,9 +17,14 @@ package org.apache.doris.datasource.paimon; +import org.apache.doris.analysis.TableSnapshot; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MTMV; import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.catalog.PartitionType; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; import org.apache.doris.datasource.CacheException; import org.apache.doris.datasource.ExternalSchemaCache; import org.apache.doris.datasource.ExternalSchemaCache.SchemaCacheKey; @@ -30,6 +35,12 @@ import org.apache.doris.datasource.mvcc.MvccUtil; import org.apache.doris.datasource.systable.SupportedSysTables; import org.apache.doris.datasource.systable.SysTable; +import org.apache.doris.mtmv.MTMVBaseTableIf; +import org.apache.doris.mtmv.MTMVRefreshContext; +import org.apache.doris.mtmv.MTMVRelatedTableIf; +import org.apache.doris.mtmv.MTMVSnapshotIdSnapshot; +import org.apache.doris.mtmv.MTMVSnapshotIf; +import org.apache.doris.mtmv.MTMVTimestampSnapshot; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ExternalAnalysisTask; @@ -38,6 +49,7 @@ import org.apache.doris.thrift.TTableType; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -58,16 +70,18 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.stream.Collectors; -public class PaimonExternalTable extends ExternalTable implements MvccTable { +public class PaimonExternalTable extends ExternalTable implements MTMVRelatedTableIf, MTMVBaseTableIf, MvccTable { private static final Logger LOG = LogManager.getLogger(PaimonExternalTable.class); - private Table paimonTable; + private final Table paimonTable; public PaimonExternalTable(long id, String name, String remoteName, PaimonExternalCatalog catalog, PaimonExternalDatabase db) { super(id, name, remoteName, catalog, db, TableType.PAIMON_EXTERNAL_TABLE); + this.paimonTable = catalog.getPaimonTable(dbName, name); } public String getPaimonCatalogType() { @@ -77,13 +91,11 @@ public String getPaimonCatalogType() { protected synchronized void makeSureInitialized() { super.makeSureInitialized(); if (!objectCreated) { - this.paimonTable = ((PaimonExternalCatalog) catalog).getPaimonTable(dbName, name); objectCreated = true; } } public Table getPaimonTable(Optional snapshot) { - makeSureInitialized(); return paimonTable.copy( Collections.singletonMap(CoreOptions.SCAN_VERSION.key(), String.valueOf(getOrFetchSnapshotCacheValue(snapshot).getSnapshot().getSnapshotId()))); @@ -143,6 +155,29 @@ public long fetchRowCount() { return rowCount > 0 ? rowCount : UNKNOWN_ROW_COUNT; } + @Override + public void beforeMTMVRefresh(MTMV mtmv) throws DdlException { + } + + @Override + public Map getAndCopyPartitionItems(Optional snapshot) { + return Maps.newHashMap(getNameToPartitionItems(snapshot)); + } + + @Override + public PartitionType getPartitionType(Optional snapshot) { + if (isPartitionInvalid(snapshot)) { + return PartitionType.UNPARTITIONED; + } + return getPartitionColumns(snapshot).size() > 0 ? PartitionType.LIST : PartitionType.UNPARTITIONED; + } + + @Override + public Set getPartitionColumnNames(Optional snapshot) { + return getPartitionColumns(snapshot).stream() + .map(c -> c.getName().toLowerCase()).collect(Collectors.toSet()); + } + @Override public List getPartitionColumns(Optional snapshot) { if (isPartitionInvalid(snapshot)) { @@ -157,7 +192,36 @@ private boolean isPartitionInvalid(Optional snapshot) { } @Override - public MvccSnapshot loadSnapshot() { + public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, + Optional snapshot) + throws AnalysisException { + PaimonPartition paimonPartition = getOrFetchSnapshotCacheValue(snapshot).getPartitionInfo().getNameToPartition() + .get(partitionName); + if (paimonPartition == null) { + throw new AnalysisException("can not find partition: " + partitionName); + } + return new MTMVTimestampSnapshot(paimonPartition.getLastUpdateTime()); + } + + @Override + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) + throws AnalysisException { + PaimonSnapshotCacheValue paimonSnapshot = getOrFetchSnapshotCacheValue(snapshot); + return new MTMVSnapshotIdSnapshot(paimonSnapshot.getSnapshot().getSnapshotId()); + } + + @Override + public boolean isPartitionColumnAllowNull() { + // Paimon will write to the 'null' partition regardless of whether it is' null or 'null'. + // The logic is inconsistent with Doris' empty partition logic, so it needs to return false. + // However, when Spark creates Paimon tables, specifying 'not null' does not take effect. + // In order to successfully create the materialized view, false is returned here. + // The cost is that Paimon partition writes a null value, and the materialized view cannot detect this data. + return true; + } + + @Override + public MvccSnapshot loadSnapshot(Optional tableSnapshot) { return new PaimonMvccSnapshot(getPaimonSnapshotCacheValue()); } @@ -201,6 +265,7 @@ public Optional initSchema(SchemaCacheKey key) { null, getCatalog().getName(), key.getDbName(), key.getTblName(), paimonSchemaCacheKey.getSchemaId()); } + } private PaimonSchema loadPaimonSchemaBySchemaId(PaimonSchemaCacheKey key) throws IOException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java index bce0e0ba73a175..c6f84ac63c6c6b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java @@ -48,6 +48,7 @@ import org.apache.doris.mtmv.MTMVRefreshEnum.MTMVState; import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; import org.apache.doris.mtmv.MTMVRefreshPartitionSnapshot; +import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.mtmv.MTMVRelation; import org.apache.doris.mtmv.MTMVUtil; import org.apache.doris.nereids.StatementContext; @@ -82,6 +83,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; @@ -203,6 +205,13 @@ public void run() throws JobException { checkColumnTypeIfChange(mtmv, ctx); } if (mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE) { + MTMVRelatedTableIf relatedTable = mtmv.getMvPartitionInfo().getRelatedTable(); + if (!relatedTable.isValidRelatedTable()) { + throw new JobException("MTMV " + mtmv.getName() + "'s related table " + relatedTable.getName() + + " is not a valid related table anymore, stop refreshing." + + " e.g. Table has multiple partition columns" + + " or including not supported transform functions."); + } MTMVPartitionUtil.alignMvPartition(mtmv); } context = MTMVRefreshContext.buildContext(mtmv); @@ -444,7 +453,7 @@ private void beforeMTMVRefresh() throws AnalysisException, DdlException { } if (tableIf instanceof MvccTable) { MvccTable mvccTable = (MvccTable) tableIf; - MvccSnapshot mvccSnapshot = mvccTable.loadSnapshot(); + MvccSnapshot mvccSnapshot = mvccTable.loadSnapshot(Optional.empty()); snapshots.put(new MvccTableInfo(mvccTable), mvccSnapshot); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionExprDateTrunc.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionExprDateTrunc.java index 95a8717e01c4c7..764c87c4b115fe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionExprDateTrunc.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionExprDateTrunc.java @@ -27,6 +27,7 @@ import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.util.PropertyAnalyzer; +import org.apache.doris.datasource.mvcc.MvccUtil; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.executable.DateTimeArithmetic; import org.apache.doris.nereids.trees.expressions.functions.executable.DateTimeExtractAndTransform; @@ -69,7 +70,7 @@ public void analyze(MTMVPartitionInfo mvPartitionInfo) throws AnalysisException String.format("timeUnit not support: %s, only support: %s", this.timeUnit, timeUnits)); } MTMVRelatedTableIf relatedTable = mvPartitionInfo.getRelatedTable(); - PartitionType partitionType = relatedTable.getPartitionType(Optional.empty()); + PartitionType partitionType = relatedTable.getPartitionType(MvccUtil.getSnapshotFromContext(relatedTable)); if (partitionType == PartitionType.RANGE) { Type partitionColumnType = MTMVPartitionUtil .getPartitionColumnType(mvPartitionInfo.getRelatedTable(), mvPartitionInfo.getRelatedCol()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionInfo.java index 7eae44db0af4cc..5a14867c7e1ab3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionInfo.java @@ -21,11 +21,11 @@ import org.apache.doris.catalog.Column; import org.apache.doris.common.AnalysisException; import org.apache.doris.datasource.CatalogMgr; +import org.apache.doris.datasource.mvcc.MvccUtil; import com.google.gson.annotations.SerializedName; import java.util.List; -import java.util.Optional; /** * MTMVPartitionInfo @@ -116,7 +116,9 @@ public int getRelatedColPos() throws AnalysisException { if (partitionType == MTMVPartitionType.SELF_MANAGE) { throw new AnalysisException("partitionType is: " + partitionType); } - List partitionColumns = getRelatedTable().getPartitionColumns(Optional.empty()); + MTMVRelatedTableIf mtmvRelatedTableIf = getRelatedTable(); + List partitionColumns = mtmvRelatedTableIf.getPartitionColumns( + MvccUtil.getSnapshotFromContext(mtmvRelatedTableIf)); for (int i = 0; i < partitionColumns.size(); i++) { if (partitionColumns.get(i).getName().equalsIgnoreCase(relatedCol)) { return i; diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java index 8da00cdeeb3421..8484b07ae0d037 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java @@ -33,6 +33,7 @@ import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; +import org.apache.doris.datasource.mvcc.MvccUtil; import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; import org.apache.doris.rpc.RpcException; @@ -320,14 +321,14 @@ public static boolean isSyncWithPartitions(MTMVRefreshContext context, String mt if (!relatedTable.needAutoRefresh()) { return true; } - // check if partitions of related table if changed + // check if partitions of related table is changed Set snapshotPartitions = mtmv.getRefreshSnapshot().getSnapshotPartitions(mtmvPartitionName); if (!Objects.equals(relatedPartitionNames, snapshotPartitions)) { return false; } for (String relatedPartitionName : relatedPartitionNames) { MTMVSnapshotIf relatedPartitionCurrentSnapshot = relatedTable - .getPartitionSnapshot(relatedPartitionName, context, Optional.empty()); + .getPartitionSnapshot(relatedPartitionName, context, MvccUtil.getSnapshotFromContext(relatedTable)); if (LOG.isDebugEnabled()) { LOG.debug(String.format("isSyncWithPartitions mvName is %s\n, mtmvPartitionName is %s\n, " + "mtmv refreshSnapshot is %s\n, relatedPartitionName is %s\n, " @@ -546,10 +547,9 @@ private static MTMVRefreshPartitionSnapshot generatePartitionSnapshot(MTMVRefres if (mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE) { MTMVRelatedTableIf relatedTable = mtmv.getMvPartitionInfo().getRelatedTable(); for (String relatedPartitionName : relatedPartitionNames) { - MTMVSnapshotIf partitionSnapshot = relatedTable - .getPartitionSnapshot(relatedPartitionName, context, Optional.empty()); - refreshPartitionSnapshot.getPartitions() - .put(relatedPartitionName, partitionSnapshot); + MTMVSnapshotIf partitionSnapshot = relatedTable.getPartitionSnapshot(relatedPartitionName, context, + MvccUtil.getSnapshotFromContext(relatedTable)); + refreshPartitionSnapshot.getPartitions().put(relatedPartitionName, partitionSnapshot); } } for (BaseTableInfo baseTableInfo : baseTables) { @@ -568,7 +568,7 @@ private static MTMVRefreshPartitionSnapshot generatePartitionSnapshot(MTMVRefres } public static Type getPartitionColumnType(MTMVRelatedTableIf relatedTable, String col) throws AnalysisException { - List partitionColumns = relatedTable.getPartitionColumns(Optional.empty()); + List partitionColumns = relatedTable.getPartitionColumns(MvccUtil.getSnapshotFromContext(relatedTable)); for (Column column : partitionColumns) { if (column.getName().equals(col)) { return column.getType(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshPartitionSnapshot.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshPartitionSnapshot.java index a8de5b6597bc49..d7d9b65f4a5f48 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshPartitionSnapshot.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshPartitionSnapshot.java @@ -100,10 +100,15 @@ public void compatible(MTMV mtmv) { } private void compatiblePartitions(MTMV mtmv) throws AnalysisException { + MTMVRelatedTableIf relatedTableIf = mtmv.getMvPartitionInfo().getRelatedTable(); + // Only olapTable has historical data issues that require compatibility + if (!(relatedTableIf instanceof OlapTable)) { + return; + } if (!checkHasDataWithoutPartitionId()) { return; } - OlapTable relatedTable = (OlapTable) mtmv.getMvPartitionInfo().getRelatedTable(); + OlapTable relatedTable = (OlapTable) relatedTableIf; for (Entry entry : partitions.entrySet()) { MTMVVersionSnapshot versionSnapshot = (MTMVVersionSnapshot) entry.getValue(); if (versionSnapshot.getId() == 0) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescInitGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescInitGenerator.java index c6b4e331184e2a..28900f38e59f62 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescInitGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescInitGenerator.java @@ -18,9 +18,9 @@ package org.apache.doris.mtmv; import org.apache.doris.common.AnalysisException; +import org.apache.doris.datasource.mvcc.MvccUtil; import java.util.Map; -import java.util.Optional; /** * get all related partition descs @@ -30,6 +30,7 @@ public class MTMVRelatedPartitionDescInitGenerator implements MTMVRelatedPartiti @Override public void apply(MTMVPartitionInfo mvPartitionInfo, Map mvProperties, RelatedPartitionDescResult lastResult) throws AnalysisException { - lastResult.setItems(mvPartitionInfo.getRelatedTable().getAndCopyPartitionItems(Optional.empty())); + MTMVRelatedTableIf relatedTable = mvPartitionInfo.getRelatedTable(); + lastResult.setItems(relatedTable.getAndCopyPartitionItems(MvccUtil.getSnapshotFromContext(relatedTable))); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescRollUpGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescRollUpGenerator.java index 325fab819d9a09..71f7fc358f5975 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescRollUpGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescRollUpGenerator.java @@ -21,6 +21,7 @@ import org.apache.doris.analysis.PartitionValue; import org.apache.doris.catalog.PartitionType; import org.apache.doris.common.AnalysisException; +import org.apache.doris.datasource.mvcc.MvccUtil; import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; import com.google.common.base.Preconditions; @@ -31,7 +32,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.Optional; import java.util.Set; /** @@ -46,7 +46,7 @@ public void apply(MTMVPartitionInfo mvPartitionInfo, Map mvPrope return; } MTMVRelatedTableIf relatedTable = mvPartitionInfo.getRelatedTable(); - PartitionType partitionType = relatedTable.getPartitionType(Optional.empty()); + PartitionType partitionType = relatedTable.getPartitionType(MvccUtil.getSnapshotFromContext(relatedTable)); if (partitionType == PartitionType.RANGE) { lastResult.setDescs(rollUpRange(lastResult.getDescs(), mvPartitionInfo)); } else if (partitionType == PartitionType.LIST) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedTableIf.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedTableIf.java index c4261aa78f10be..821595d7dffd06 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedTableIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedTableIf.java @@ -115,4 +115,13 @@ default boolean needAutoRefresh() { * @return */ boolean isPartitionColumnAllowNull(); + + /** + * If the table is supported as related table. + * For example, an Iceberg table may become unsupported after partition revolution. + * @return + */ + default boolean isValidRelatedTable() { + return true; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVSnapshotIdSnapshot.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVSnapshotIdSnapshot.java new file mode 100644 index 00000000000000..cad23795c32cda --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVSnapshotIdSnapshot.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.mtmv; + +import com.google.common.base.Objects; +import com.google.gson.annotations.SerializedName; + +/** + * use snapshotId as identification of data changes + */ +public class MTMVSnapshotIdSnapshot implements MTMVSnapshotIf { + @SerializedName("s") + private long snapshotId; + + public MTMVSnapshotIdSnapshot(long snapshotId) { + this.snapshotId = snapshotId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MTMVSnapshotIdSnapshot that = (MTMVSnapshotIdSnapshot) o; + return snapshotId == that.snapshotId; + } + + @Override + public int hashCode() { + return Objects.hashCode(snapshotId); + } + + @Override + public String toString() { + return "MTMVSnapshotIdSnapshot{" + + "snapshotId=" + snapshotId + + '}'; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVVersionSnapshot.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVVersionSnapshot.java index 2440649462ebf3..9ca95caebdd4b2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVVersionSnapshot.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVVersionSnapshot.java @@ -31,10 +31,6 @@ public class MTMVVersionSnapshot implements MTMVSnapshotIf { @SerializedName("id") private long id; - public MTMVVersionSnapshot(long version) { - this.version = version; - } - public MTMVVersionSnapshot(long version, long id) { this.version = version; this.id = id; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java index 0572b89340dd3a..5faadd0b271169 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java @@ -223,7 +223,6 @@ private Plan planWithLock(LogicalPlan plan, PhysicalProperties requireProperties // collect table and lock them in the order of table id collectAndLockTable(showAnalyzeProcess(explainLevel, showPlanProcess)); // after table collector, we should use a new context. - statementContext.loadSnapshots(); Plan resultPlan = planWithoutLock(plan, requireProperties, explainLevel, showPlanProcess); lockCallback.accept(resultPlan); if (statementContext.getConnectContext().getExecutor() != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java index a1d30e998e863a..ec58756d3e62a9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids; import org.apache.doris.analysis.StatementBase; +import org.apache.doris.analysis.TableSnapshot; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.View; @@ -648,13 +649,13 @@ public void addPlannerHook(PlannerHook plannerHook) { /** * Load snapshot information of mvcc */ - public void loadSnapshots() { + public void loadSnapshots(Optional tableSnapshot) { for (TableIf tableIf : tables.values()) { if (tableIf instanceof MvccTable) { MvccTableInfo mvccTableInfo = new MvccTableInfo(tableIf); // may be set by MTMV, we can not load again if (!snapshots.containsKey(mvccTableInfo)) { - snapshots.put(mvccTableInfo, ((MvccTable) tableIf).loadSnapshot()); + snapshots.put(mvccTableInfo, ((MvccTable) tableIf).loadSnapshot(tableSnapshot)); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index 3e28720b8f8eab..c99980bda957a3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -382,6 +382,7 @@ private LogicalPlan getLogicalPlan(TableIf table, UnboundRelation unboundRelatio List qualifierWithoutTableName = Lists.newArrayList(); qualifierWithoutTableName.addAll(qualifiedTableName.subList(0, qualifiedTableName.size() - 1)); + cascadesContext.getStatementContext().loadSnapshots(unboundRelation.getTableSnapshot()); boolean isView = false; try { switch (table.getType()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java index ea94c5ef9679c4..ad448e25849b94 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java @@ -464,17 +464,14 @@ protected Pair>, Map>> return Pair.of(ImmutableMap.of(), ImmutableMap.of()); } // Collect the mv related base table partitions which query used - Map> queryUsedBaseTablePartitions = new LinkedHashMap<>(); + Map> queryUsedBaseTablePartitions = new LinkedHashMap<>(); queryUsedBaseTablePartitions.put(relatedPartitionTable, new HashSet<>()); queryPlan.accept(new StructInfo.QueryScanPartitionsCollector(), queryUsedBaseTablePartitions); // Bail out, not check invalid partition if not olap scan, support later if (queryUsedBaseTablePartitions.isEmpty()) { return Pair.of(ImmutableMap.of(), ImmutableMap.of()); } - Set queryUsedBaseTablePartitionNameSet = queryUsedBaseTablePartitions.get(relatedPartitionTable) - .stream() - .map(Partition::getName) - .collect(Collectors.toSet()); + Set queryUsedBaseTablePartitionNameSet = queryUsedBaseTablePartitions.get(relatedPartitionTable); Collection mvValidPartitions = MTMVRewriteUtil.getMTMVCanRewritePartitions(mtmv, cascadesContext.getConnectContext(), System.currentTimeMillis(), false); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java index 24f4c241b5399e..8a61d56646d812 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java @@ -24,6 +24,7 @@ import org.apache.doris.catalog.PartitionType; import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.constraint.TableIdentifier; +import org.apache.doris.datasource.mvcc.MvccUtil; import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.nereids.CascadesContext; @@ -450,13 +451,14 @@ public Void visitLogicalRelation(LogicalRelation relation, IncrementCheckerConte return null; } MTMVRelatedTableIf relatedTable = (MTMVRelatedTableIf) table; - PartitionType type = relatedTable.getPartitionType(Optional.empty()); + PartitionType type = relatedTable.getPartitionType(MvccUtil.getSnapshotFromContext(relatedTable)); if (PartitionType.UNPARTITIONED.equals(type)) { context.addFailReason(String.format("related base table is not partition table, the table is %s", table.getName())); return null; } - Set partitionColumnSet = new HashSet<>(relatedTable.getPartitionColumns(Optional.empty())); + Set partitionColumnSet = new HashSet<>( + relatedTable.getPartitionColumns(MvccUtil.getSnapshotFromContext(relatedTable))); Column mvReferenceColumn = contextPartitionColumn.getOriginalColumn().get(); Expr definExpr = mvReferenceColumn.getDefineExpr(); if (definExpr instanceof SlotRef) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java index e9b495f08a3921..3f0397dc4115ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java @@ -17,9 +17,9 @@ package org.apache.doris.nereids.rules.exploration.mv; -import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.TableIf; import org.apache.doris.common.Pair; +import org.apache.doris.datasource.ExternalTable; import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.jobs.executor.Rewriter; @@ -51,6 +51,8 @@ import org.apache.doris.nereids.trees.plans.commands.UpdateMvByPartitionCommand.PredicateAdder; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan.SelectedPartitions; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; @@ -768,22 +770,28 @@ public Plan visitLogicalOlapScan(LogicalOlapScan olapScan, * Collect partitions on base table */ public static class QueryScanPartitionsCollector extends DefaultPlanVisitor>> { + Map>> { @Override public Plan visitLogicalCatalogRelation(LogicalCatalogRelation catalogRelation, - Map> targetTablePartitionMap) { + Map> targetTablePartitionMap) { TableIf table = catalogRelation.getTable(); BaseTableInfo relatedPartitionTable = new BaseTableInfo(table); if (!targetTablePartitionMap.containsKey(relatedPartitionTable)) { return catalogRelation; } + Set tablePartitions = targetTablePartitionMap.get(relatedPartitionTable); if (catalogRelation instanceof LogicalOlapScan) { // Handle olap table LogicalOlapScan logicalOlapScan = (LogicalOlapScan) catalogRelation; - Set tablePartitions = targetTablePartitionMap.get(relatedPartitionTable); for (Long partitionId : logicalOlapScan.getSelectedPartitionIds()) { - tablePartitions.add(logicalOlapScan.getTable().getPartition(partitionId)); + tablePartitions.add(logicalOlapScan.getTable().getPartition(partitionId).getName()); } + } else if (catalogRelation instanceof LogicalFileScan + && catalogRelation.getTable() instanceof ExternalTable + && ((ExternalTable) catalogRelation.getTable()).supportInternalPartitionPruned()) { + LogicalFileScan logicalFileScan = (LogicalFileScan) catalogRelation; + SelectedPartitions selectedPartitions = logicalFileScan.getSelectedPartitions(); + tablePartitions.addAll(selectedPartitions.selectedPartitions.keySet()); } else { // todo Support other type partition table // Not support to partition check now when query external catalog table, support later. diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java index d35eafffa2c21c..fac66f97abb50c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java @@ -315,13 +315,9 @@ public Plan visitLogicalCatalogRelation(LogicalCatalogRelation catalogRelation, MTMVRelatedTableIf targetTable = (MTMVRelatedTableIf) table; for (String partitionName : filterTableEntry.getValue()) { Partition partition = targetTable.getPartition(partitionName); - if (!(targetTable instanceof OlapTable)) { - // check partition is have data or not, only support olap table - break; - } - if (!((OlapTable) targetTable).selectNonEmptyPartitionIds( + if (targetTable instanceof OlapTable && !((OlapTable) targetTable).selectNonEmptyPartitionIds( Lists.newArrayList(partition.getId())).isEmpty()) { - // Add filter only when partition has data + // Add filter only when partition has data when olap table partitionHasDataItems.add( ((OlapTable) targetTable).getPartitionInfo().getItem(partition.getId())); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java index c2e9abd2f0f97c..8624bed9ceefae 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java @@ -25,8 +25,8 @@ import org.apache.doris.analysis.FunctionParams; import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.StringLiteral; +import org.apache.doris.catalog.PartitionType; import org.apache.doris.common.DdlException; -import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.mtmv.MTMVPartitionExprFactory; import org.apache.doris.mtmv.MTMVPartitionInfo; import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; @@ -136,9 +136,9 @@ private RelatedTableInfo getRelatedTableInfo(NereidsPlanner planner, String part if (!partitionColumnNames.contains(relatedTableInfo.getColumn())) { throw new AnalysisException("error related column: " + relatedTableInfo.getColumn()); } - if (!(mtmvBaseRelatedTable instanceof HMSExternalTable) + if (!(mtmvBaseRelatedTable.getPartitionType(Optional.empty()).equals(PartitionType.LIST)) && partitionColumnNames.size() != 1) { - throw new AnalysisException("only hms table support multi column partition."); + throw new AnalysisException("only List PartitionType support multi column partition."); } return relatedTableInfo; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java index 9f0a35cdc10a9c..1a7d3e4eff3d7c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java @@ -202,6 +202,7 @@ import org.apache.doris.load.sync.SyncJob; import org.apache.doris.load.sync.canal.CanalSyncJob; import org.apache.doris.mtmv.MTMVMaxTimestampSnapshot; +import org.apache.doris.mtmv.MTMVSnapshotIdSnapshot; import org.apache.doris.mtmv.MTMVSnapshotIf; import org.apache.doris.mtmv.MTMVTimestampSnapshot; import org.apache.doris.mtmv.MTMVVersionSnapshot; @@ -450,6 +451,7 @@ public class GsonUtils { RuntimeTypeAdapterFactory.of(MTMVSnapshotIf.class, "clazz") .registerSubtype(MTMVMaxTimestampSnapshot.class, MTMVMaxTimestampSnapshot.class.getSimpleName()) .registerSubtype(MTMVTimestampSnapshot.class, MTMVTimestampSnapshot.class.getSimpleName()) + .registerSubtype(MTMVSnapshotIdSnapshot.class, MTMVSnapshotIdSnapshot.class.getSimpleName()) .registerSubtype(MTMVVersionSnapshot.class, MTMVVersionSnapshot.class.getSimpleName()); private static RuntimeTypeAdapterFactory dbTypeAdapterFactory = RuntimeTypeAdapterFactory.of( diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java index 41afbc1714f85a..b54549af69bb87 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java @@ -139,6 +139,8 @@ import org.apache.doris.catalog.MetadataViewer; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.catalog.PartitionKey; import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.ReplicaAllocation; @@ -203,6 +205,7 @@ import org.apache.doris.datasource.hive.HiveMetaStoreClientHelper; import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; import org.apache.doris.datasource.iceberg.IcebergExternalDatabase; +import org.apache.doris.datasource.iceberg.IcebergExternalTable; import org.apache.doris.datasource.maxcompute.MaxComputeExternalCatalog; import org.apache.doris.job.manager.JobManager; import org.apache.doris.load.DeleteHandler; @@ -256,6 +259,7 @@ import com.google.common.base.Strings; import com.google.common.collect.ImmutableSetMultimap; import com.google.common.collect.Lists; +import com.google.common.collect.Range; import com.google.common.collect.Sets; import com.google.gson.GsonBuilder; import org.apache.commons.collections.CollectionUtils; @@ -1898,6 +1902,8 @@ private void handleShowPartitions() throws AnalysisException { resultSet = new ShowResultSet(showStmt.getMetaData(), rows); } else if (showStmt.getCatalog() instanceof MaxComputeExternalCatalog) { handleShowMaxComputeTablePartitions(showStmt); + } else if (showStmt.getCatalog() instanceof IcebergExternalCatalog) { + handleShowIcebergTablePartitions(showStmt); } else { handleShowHMSTablePartitions(showStmt); } @@ -1981,6 +1987,40 @@ private void handleShowHMSTablePartitions(ShowPartitionsStmt showStmt) throws An resultSet = new ShowResultSet(showStmt.getMetaData(), rows); } + private void handleShowIcebergTablePartitions(ShowPartitionsStmt showStmt) { + IcebergExternalCatalog catalog = (IcebergExternalCatalog) showStmt.getCatalog(); + String db = showStmt.getTableName().getDb(); + String tbl = showStmt.getTableName().getTbl(); + IcebergExternalTable icebergTable = (IcebergExternalTable) catalog.getDb(db).get().getTable(tbl).get(); + LimitElement limit = showStmt.getLimitElement(); + List orderByPairs = showStmt.getOrderByPairs(); + Map partitions = icebergTable.getAndCopyPartitionItems(Optional.empty()); + List> rows = new ArrayList<>(); + for (Map.Entry entry : partitions.entrySet()) { + List row = new ArrayList<>(); + Range items = entry.getValue().getItems(); + row.add(entry.getKey()); + row.add(items.lowerEndpoint().toString()); + row.add(items.upperEndpoint().toString()); + rows.add(row); + } + // sort by partition name + if (orderByPairs != null && orderByPairs.get(0).isDesc()) { + rows.sort(Comparator.comparing(x -> x.get(0), Comparator.reverseOrder())); + } else { + rows.sort(Comparator.comparing(x -> x.get(0))); + } + if (limit != null && limit.hasLimit()) { + int beginIndex = (int) limit.getOffset(); + int endIndex = (int) (beginIndex + limit.getLimit()); + if (endIndex > rows.size()) { + endIndex = rows.size(); + } + rows = rows.subList(beginIndex, endIndex); + } + resultSet = new ShowResultSet(showStmt.getMetaData(), rows); + } + private void handleShowTablet() throws AnalysisException { ShowTabletStmt showStmt = (ShowTabletStmt) stmt; List> rows = Lists.newArrayList(); 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 d3b8529c3c1bac..cebda862a83d49 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 @@ -63,6 +63,7 @@ import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; import org.apache.doris.datasource.iceberg.IcebergMetadataCache; import org.apache.doris.datasource.maxcompute.MaxComputeExternalCatalog; +import org.apache.doris.datasource.mvcc.MvccUtil; import org.apache.doris.job.common.JobType; import org.apache.doris.job.extensions.mtmv.MTMVJob; import org.apache.doris.job.task.AbstractTask; @@ -1661,7 +1662,7 @@ private static List partitionValuesMetadataResultForHmsTable(HMSExternalTa HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() .getMetaStoreCache((HMSExternalCatalog) tbl.getCatalog()); HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( - tbl.getDbName(), tbl.getName(), tbl.getPartitionColumnTypes()); + tbl.getDbName(), tbl.getName(), tbl.getPartitionColumnTypes(MvccUtil.getSnapshotFromContext(tbl))); Map> valuesMap = hivePartitionValues.getPartitionValuesMap(); List dataBatch = Lists.newArrayList(); for (Map.Entry> entry : valuesMap.entrySet()) { 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 759653e753980f..409fc1daf725cc 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 @@ -175,7 +175,7 @@ public org.apache.hadoop.hive.metastore.api.Table getRemoteTable() { HMSExternalCatalog catalog = new HMSExternalCatalog(); HMSExternalDatabase db = new HMSExternalDatabase(catalog, 1, "db", "db"); HMSExternalTable hmsExternalTable = new HMSExternalTable(2, "tb", "tb", catalog, db); - HiveMetaStoreClientHelper.getHudiTableSchema(hmsExternalTable); + HiveMetaStoreClientHelper.getHudiTableSchema(hmsExternalTable, new boolean[] {false}, "20241219214518880"); // 4. delete the commit file, // this operation is used to imitate the clean operation in hudi @@ -189,7 +189,7 @@ public org.apache.hadoop.hive.metastore.api.Table getRemoteTable() { // because we will refresh timeline in this `getHudiTableSchema` method, // and we can get the latest commit. // so that this error: `Could not read commit details from file /.hoodie/1.commit` will be not reported. - HiveMetaStoreClientHelper.getHudiTableSchema(hmsExternalTable); + HiveMetaStoreClientHelper.getHudiTableSchema(hmsExternalTable, new boolean[] {false}, "20241219214518880"); // 7. clean up Assert.assertTrue(commit2.delete()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergExternalTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergExternalTableTest.java new file mode 100644 index 00000000000000..1032d2ce58f221 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergExternalTableTest.java @@ -0,0 +1,23 @@ +// 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; + +public class IcebergExternalTableTest { + + // branch3.1 need pick by other pr +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergPartitionInfoTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergPartitionInfoTest.java new file mode 100644 index 00000000000000..74c8c3f6954a97 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergPartitionInfoTest.java @@ -0,0 +1,53 @@ +// 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 com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Map; +import java.util.Set; + +public class IcebergPartitionInfoTest { + + @Test + public void testGetLatestSnapshotId() { + IcebergPartition p1 = new IcebergPartition("p1", 0, 0, 0, 0, 1, 101, null, null); + IcebergPartition p2 = new IcebergPartition("p2", 0, 0, 0, 0, 2, 102, null, null); + IcebergPartition p3 = new IcebergPartition("p3", 0, 0, 0, 0, 3, 103, null, null); + Map nameToIcebergPartition = Maps.newHashMap(); + nameToIcebergPartition.put(p1.getPartitionName(), p1); + nameToIcebergPartition.put(p2.getPartitionName(), p2); + nameToIcebergPartition.put(p3.getPartitionName(), p3); + Map> nameToIcebergPartitionNames = Maps.newHashMap(); + Set names = Sets.newHashSet(); + names.add("p1"); + names.add("p2"); + nameToIcebergPartitionNames.put("p1", names); + + IcebergPartitionInfo info = new IcebergPartitionInfo(null, nameToIcebergPartition, nameToIcebergPartitionNames); + long snapshot1 = info.getLatestSnapshotId("p1"); + long snapshot2 = info.getLatestSnapshotId("p2"); + long snapshot3 = info.getLatestSnapshotId("p3"); + Assertions.assertEquals(102, snapshot1); + Assertions.assertEquals(102, snapshot2); + Assertions.assertEquals(103, snapshot3); + } +} 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 85c4d0be6c1924..583b637746e338 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 @@ -29,11 +29,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.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.HiveDlaTable; import org.apache.doris.nereids.datasets.tpch.AnalyzeCheckTestBase; import org.apache.doris.qe.SessionVariable; @@ -107,6 +109,7 @@ private void createDbAndTableForHmsCatalog(HMSExternalCatalog hmsCatalog) { Deencapsulation.setField(tbl, "catalog", hmsCatalog); Deencapsulation.setField(tbl, "dbName", "hms_db"); Deencapsulation.setField(tbl, "name", "hms_tbl"); + Deencapsulation.setField(tbl, "dlaTable", new HiveDlaTable(tbl)); Deencapsulation.setField(tbl, "dlaType", DLAType.HIVE); new Expectations(tbl) { { @@ -139,7 +142,7 @@ private void createDbAndTableForHmsCatalog(HMSExternalCatalog hmsCatalog) { result = TableIf.TableType.HMS_EXTERNAL_TABLE; // mock initSchemaAndUpdateTime and do nothing - tbl.initSchemaAndUpdateTime(); + tbl.initSchemaAndUpdateTime(new ExternalSchemaCache.SchemaCacheKey("hms_db", "hms_tbl")); minTimes = 0; tbl.getDatabase(); @@ -211,7 +214,6 @@ private void createDbAndTableForHmsCatalog(HMSExternalCatalog hmsCatalog) { Deencapsulation.setField(view2, "dbName", "hms_db"); Deencapsulation.setField(view2, "name", "hms_view2"); Deencapsulation.setField(view2, "dlaType", DLAType.HIVE); - new Expectations(view2) { { @@ -264,7 +266,6 @@ private void createDbAndTableForHmsCatalog(HMSExternalCatalog hmsCatalog) { Deencapsulation.setField(view3, "dbName", "hms_db"); Deencapsulation.setField(view3, "name", "hms_view3"); Deencapsulation.setField(view3, "dlaType", DLAType.HIVE); - new Expectations(view3) { { @@ -317,7 +318,6 @@ private void createDbAndTableForHmsCatalog(HMSExternalCatalog hmsCatalog) { Deencapsulation.setField(view4, "dbName", "hms_db"); Deencapsulation.setField(view4, "name", "hms_view4"); Deencapsulation.setField(view4, "dlaType", DLAType.HIVE); - new Expectations(view4) { { diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRefreshSnapshotTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRefreshSnapshotTest.java index 1890f9c9805926..3e61a7ca54aa1f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRefreshSnapshotTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRefreshSnapshotTest.java @@ -35,8 +35,8 @@ public class MTMVRefreshSnapshotTest { private long baseExistTableId = 1L; private long correctVersion = 1L; private MTMVRefreshSnapshot refreshSnapshot = new MTMVRefreshSnapshot(); - private MTMVVersionSnapshot p1Snapshot = new MTMVVersionSnapshot(correctVersion); - private MTMVVersionSnapshot t1Snapshot = new MTMVVersionSnapshot(correctVersion); + private MTMVVersionSnapshot p1Snapshot = new MTMVVersionSnapshot(correctVersion, 0); + private MTMVVersionSnapshot t1Snapshot = new MTMVVersionSnapshot(correctVersion, 0); @Mocked private BaseTableInfo existTable; @Mocked @@ -92,19 +92,19 @@ public void setUp() throws NoSuchMethodException, SecurityException, AnalysisExc public void testPartitionSync() { // normal boolean sync = refreshSnapshot.equalsWithRelatedPartition(mvExistPartitionName, relatedExistPartitionName, - new MTMVVersionSnapshot(correctVersion)); + new MTMVVersionSnapshot(correctVersion, 0)); Assert.assertTrue(sync); // non exist mv partition sync = refreshSnapshot.equalsWithRelatedPartition("mvp2", relatedExistPartitionName, - new MTMVVersionSnapshot(correctVersion)); + new MTMVVersionSnapshot(correctVersion, 0)); Assert.assertFalse(sync); // non exist related partition sync = refreshSnapshot - .equalsWithRelatedPartition(mvExistPartitionName, "p2", new MTMVVersionSnapshot(correctVersion)); + .equalsWithRelatedPartition(mvExistPartitionName, "p2", new MTMVVersionSnapshot(correctVersion, 0)); Assert.assertFalse(sync); // snapshot value not equal sync = refreshSnapshot.equalsWithRelatedPartition(mvExistPartitionName, relatedExistPartitionName, - new MTMVVersionSnapshot(2L)); + new MTMVVersionSnapshot(2L, 0)); Assert.assertFalse(sync); // snapshot type not equal sync = refreshSnapshot.equalsWithRelatedPartition(mvExistPartitionName, relatedExistPartitionName, @@ -116,19 +116,19 @@ public void testPartitionSync() { public void testTableSync() { // normal boolean sync = refreshSnapshot.equalsWithBaseTable(mvExistPartitionName, existTable, - new MTMVVersionSnapshot(correctVersion)); + new MTMVVersionSnapshot(correctVersion, 0)); Assert.assertTrue(sync); // non exist mv partition sync = refreshSnapshot - .equalsWithBaseTable("mvp2", existTable, new MTMVVersionSnapshot(correctVersion)); + .equalsWithBaseTable("mvp2", existTable, new MTMVVersionSnapshot(correctVersion, 0)); Assert.assertFalse(sync); // non exist related partition sync = refreshSnapshot - .equalsWithBaseTable(mvExistPartitionName, nonExistTable, new MTMVVersionSnapshot(correctVersion)); + .equalsWithBaseTable(mvExistPartitionName, nonExistTable, new MTMVVersionSnapshot(correctVersion, 0)); Assert.assertFalse(sync); // snapshot value not equal sync = refreshSnapshot - .equalsWithBaseTable(mvExistPartitionName, existTable, new MTMVVersionSnapshot(2L)); + .equalsWithBaseTable(mvExistPartitionName, existTable, new MTMVVersionSnapshot(2L, 0)); Assert.assertFalse(sync); // snapshot type not equal sync = refreshSnapshot.equalsWithBaseTable(mvExistPartitionName, existTable, 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 b749d87652bc64..8c530908fb438b 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 @@ -31,10 +31,12 @@ 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.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.HiveDlaTable; import org.apache.doris.datasource.hive.source.HiveScanNode; import org.apache.doris.datasource.systable.SupportedSysTables; import org.apache.doris.nereids.datasets.tpch.AnalyzeCheckTestBase; @@ -124,6 +126,7 @@ private void init(HMSExternalCatalog hmsCatalog) { Deencapsulation.setField(tbl, "catalog", hmsCatalog); Deencapsulation.setField(tbl, "dbName", "hms_db"); Deencapsulation.setField(tbl, "name", "hms_tbl"); + Deencapsulation.setField(tbl, "dlaTable", new HiveDlaTable(tbl)); Deencapsulation.setField(tbl, "dlaType", DLAType.HIVE); new Expectations(tbl) { { @@ -160,7 +163,7 @@ private void init(HMSExternalCatalog hmsCatalog) { result = DLAType.HIVE; // mock initSchemaAndUpdateTime and do nothing - tbl.initSchemaAndUpdateTime(); + tbl.initSchemaAndUpdateTime(new ExternalSchemaCache.SchemaCacheKey("hms_db", "hms_tbl")); minTimes = 0; tbl.getDatabase(); @@ -175,6 +178,7 @@ private void init(HMSExternalCatalog hmsCatalog) { Deencapsulation.setField(tbl2, "catalog", hmsCatalog); Deencapsulation.setField(tbl2, "dbName", "hms_db"); Deencapsulation.setField(tbl2, "name", "hms_tbl2"); + Deencapsulation.setField(tbl2, "dlaTable", new HiveDlaTable(tbl2)); Deencapsulation.setField(tbl, "dlaType", DLAType.HIVE); new Expectations(tbl2) { { @@ -211,7 +215,7 @@ private void init(HMSExternalCatalog hmsCatalog) { result = DLAType.HIVE; // mock initSchemaAndUpdateTime and do nothing - tbl2.initSchemaAndUpdateTime(); + tbl2.initSchemaAndUpdateTime(new ExternalSchemaCache.SchemaCacheKey("hms_db", "hms_tbl2")); minTimes = 0; tbl2.getDatabase(); @@ -401,7 +405,7 @@ public void testHitSqlCacheAfterPartitionUpdateTimeChanged() throws Exception { List scanNodes = Arrays.asList(hiveScanNode4); // invoke initSchemaAndUpdateTime first and init schemaUpdateTime - tbl2.initSchemaAndUpdateTime(); + tbl2.initSchemaAndUpdateTime(new ExternalSchemaCache.SchemaCacheKey(tbl2.getDbName(), tbl2.getName())); CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheMode(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); @@ -449,7 +453,7 @@ public void testHitSqlCacheByNereidsAfterPartitionUpdateTimeChanged() { List scanNodes = Arrays.asList(hiveScanNode4); // invoke initSchemaAndUpdateTime first and init schemaUpdateTime - tbl2.initSchemaAndUpdateTime(); + tbl2.initSchemaAndUpdateTime(new ExternalSchemaCache.SchemaCacheKey(tbl2.getDbName(), tbl2.getName())); CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheModeForNereids(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); diff --git a/regression-test/data/external_table_p2/hudi/hudi_mtmv/test_hudi_mtmv.out b/regression-test/data/external_table_p2/hudi/hudi_mtmv/test_hudi_mtmv.out new file mode 100644 index 00000000000000..f8870d0b2876a3 --- /dev/null +++ b/regression-test/data/external_table_p2/hudi/hudi_mtmv/test_hudi_mtmv.out @@ -0,0 +1,90 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !base_table -- +20250121171615893 20250121171615893_0_0 20250121171615893_3_1 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 5 5 a +20250121171615893 20250121171615893_0_1 20250121171615893_5_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 7 7 a +20250121171615893 20250121171615893_0_2 20250121171615893_7_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 9 9 a +20250121171615893 20250121171615893_0_3 20250121171615893_0_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 1 1 a +20250121171615893 20250121171615893_0_4 20250121171615893_2_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 3 3 a +20250121171615893 20250121171615893_1_0 20250121171615893_4_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 6 6 b +20250121171615893 20250121171615893_1_1 20250121171615893_6_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 8 8 b +20250121171615893 20250121171615893_1_2 20250121171615893_1_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 2 2 b +20250121171615893 20250121171615893_1_3 20250121171615893_3_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 4 4 b +20250121171615893 20250121171615893_1_4 20250121171615893_7_1 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 10 10 b + +-- !refresh_one_partition -- +20250121171615893 20250121171615893_0_0 20250121171615893_3_1 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 5 5 a +20250121171615893 20250121171615893_0_1 20250121171615893_5_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 7 7 a +20250121171615893 20250121171615893_0_2 20250121171615893_7_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 9 9 a +20250121171615893 20250121171615893_0_3 20250121171615893_0_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 1 1 a +20250121171615893 20250121171615893_0_4 20250121171615893_2_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 3 3 a + +-- !refresh_auto -- +20250121171615893 20250121171615893_0_0 20250121171615893_3_1 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 5 5 a +20250121171615893 20250121171615893_0_1 20250121171615893_5_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 7 7 a +20250121171615893 20250121171615893_0_2 20250121171615893_7_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 9 9 a +20250121171615893 20250121171615893_0_3 20250121171615893_0_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 1 1 a +20250121171615893 20250121171615893_0_4 20250121171615893_2_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 3 3 a +20250121171615893 20250121171615893_1_0 20250121171615893_4_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 6 6 b +20250121171615893 20250121171615893_1_1 20250121171615893_6_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 8 8 b +20250121171615893 20250121171615893_1_2 20250121171615893_1_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 2 2 b +20250121171615893 20250121171615893_1_3 20250121171615893_3_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 4 4 b +20250121171615893 20250121171615893_1_4 20250121171615893_7_1 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 10 10 b + +-- !is_sync_before_rebuild -- +true + +-- !is_sync_after_rebuild -- +true + +-- !refresh_complete_rebuild -- +20250121171615893 20250121171615893_0_0 20250121171615893_3_1 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 5 5 a +20250121171615893 20250121171615893_0_1 20250121171615893_5_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 7 7 a +20250121171615893 20250121171615893_0_2 20250121171615893_7_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 9 9 a +20250121171615893 20250121171615893_0_3 20250121171615893_0_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 1 1 a +20250121171615893 20250121171615893_0_4 20250121171615893_2_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 3 3 a +20250121171615893 20250121171615893_1_0 20250121171615893_4_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 6 6 b +20250121171615893 20250121171615893_1_1 20250121171615893_6_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 8 8 b +20250121171615893 20250121171615893_1_2 20250121171615893_1_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 2 2 b +20250121171615893 20250121171615893_1_3 20250121171615893_3_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 4 4 b +20250121171615893 20250121171615893_1_4 20250121171615893_7_1 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 10 10 b + +-- !not_partition_before -- +false + +-- !not_partition -- +1 1 a +10 10 b +2 2 b +3 3 a +4 4 b +5 5 a +6 6 b +7 7 a +8 8 b +9 9 a + +-- !not_partition_after -- +true + +-- !join_one_partition -- +20250121171615893 20250121171615893_0_0 20250121171615893_3_1 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 5 5 a \N \N +20250121171615893 20250121171615893_0_1 20250121171615893_5_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 7 7 a \N \N +20250121171615893 20250121171615893_0_2 20250121171615893_7_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 9 9 a \N \N +20250121171615893 20250121171615893_0_3 20250121171615893_0_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 1 1 a 1 2 +20250121171615893 20250121171615893_0_4 20250121171615893_2_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 3 3 a \N \N + +-- !two_partition -- +20250121172644680 20250121172644680_0_0 20250121172644680_1_0 create_date=2020-01-01/country=sh f476d3de-4f54-46c2-a674-298ba7655878-0_0-75-218_20250121172644680.parquet 2 2020-01-01 sh +20250121172644680 20250121172644680_1_0 20250121172644680_4_0 create_date=2038-01-02/country=bj 830d5f76-f546-42e3-917c-4d64fa5d99cb-0_1-75-217_20250121172644680.parquet 5 2038-01-02 bj +20250121172644680 20250121172644680_2_0 20250121172644680_3_0 create_date=2038-01-01/country=sh 67d1c440-b1b0-4a1d-81a3-5c6d1dbb5995-0_2-75-220_20250121172644680.parquet 4 2038-01-01 sh +20250121172644680 20250121172644680_3_0 20250121172644680_2_0 create_date=2038-01-01/country=bj b8746f01-4f34-43d2-863a-c2d8701eadff-0_3-75-219_20250121172644680.parquet 3 2038-01-01 bj +20250121172644680 20250121172644680_4_0 20250121172644680_0_0 create_date=2020-01-01/country=bj 4c8f5ef5-58a3-4a87-a811-8a0365d1adcd-0_4-75-221_20250121172644680.parquet 1 2020-01-01 bj + +-- !limit_partition -- +20250121172644680 20250121172644680_1_0 20250121172644680_4_0 create_date=2038-01-02/country=bj 830d5f76-f546-42e3-917c-4d64fa5d99cb-0_1-75-217_20250121172644680.parquet 5 2038-01-02 bj +20250121172644680 20250121172644680_2_0 20250121172644680_3_0 create_date=2038-01-01/country=sh 67d1c440-b1b0-4a1d-81a3-5c6d1dbb5995-0_2-75-220_20250121172644680.parquet 4 2038-01-01 sh +20250121172644680 20250121172644680_3_0 20250121172644680_2_0 create_date=2038-01-01/country=bj b8746f01-4f34-43d2-863a-c2d8701eadff-0_3-75-219_20250121172644680.parquet 3 2038-01-01 bj + +-- !null_partition -- +20250121173026848 20250121173026848_0_0 20250121173026848_0_0 region=bj 5e4042ea-b5e9-4a87-8a1c-99605d1c2b4f-0_0-96-286_20250121173026848.parquet 1 bj + diff --git a/regression-test/data/external_table_p2/hudi/hudi_mtmv/test_hudi_olap_rewrite_mtmv.out b/regression-test/data/external_table_p2/hudi/hudi_mtmv/test_hudi_olap_rewrite_mtmv.out new file mode 100644 index 00000000000000..30b64a98ad8db8 --- /dev/null +++ b/regression-test/data/external_table_p2/hudi/hudi_mtmv/test_hudi_olap_rewrite_mtmv.out @@ -0,0 +1,44 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !refresh_one_partition -- +20250121171615893 20250121171615893_0_0 20250121171615893_3_1 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 5 5 a \N \N +20250121171615893 20250121171615893_0_1 20250121171615893_5_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 7 7 a \N \N +20250121171615893 20250121171615893_0_2 20250121171615893_7_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 9 9 a \N \N +20250121171615893 20250121171615893_0_3 20250121171615893_0_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 1 1 a 1 2 +20250121171615893 20250121171615893_0_4 20250121171615893_2_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 3 3 a \N \N + +-- !refresh_one_partition_rewrite -- +20250121171615893 20250121171615893_0_0 20250121171615893_3_1 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 5 5 a \N \N +20250121171615893 20250121171615893_0_1 20250121171615893_5_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 7 7 a \N \N +20250121171615893 20250121171615893_0_2 20250121171615893_7_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 9 9 a \N \N +20250121171615893 20250121171615893_0_3 20250121171615893_0_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 1 1 a 1 2 +20250121171615893 20250121171615893_0_4 20250121171615893_2_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 3 3 a \N \N +20250121171615893 20250121171615893_1_0 20250121171615893_4_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 6 6 b \N \N +20250121171615893 20250121171615893_1_1 20250121171615893_6_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 8 8 b \N \N +20250121171615893 20250121171615893_1_2 20250121171615893_1_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 2 2 b \N \N +20250121171615893 20250121171615893_1_3 20250121171615893_3_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 4 4 b \N \N +20250121171615893 20250121171615893_1_4 20250121171615893_7_1 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 10 10 b \N \N + +-- !refresh_auto -- +20250121171615893 20250121171615893_0_0 20250121171615893_3_1 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 5 5 a \N \N +20250121171615893 20250121171615893_0_1 20250121171615893_5_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 7 7 a \N \N +20250121171615893 20250121171615893_0_2 20250121171615893_7_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 9 9 a \N \N +20250121171615893 20250121171615893_0_3 20250121171615893_0_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 1 1 a 1 2 +20250121171615893 20250121171615893_0_4 20250121171615893_2_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 3 3 a \N \N +20250121171615893 20250121171615893_1_0 20250121171615893_4_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 6 6 b \N \N +20250121171615893 20250121171615893_1_1 20250121171615893_6_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 8 8 b \N \N +20250121171615893 20250121171615893_1_2 20250121171615893_1_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 2 2 b \N \N +20250121171615893 20250121171615893_1_3 20250121171615893_3_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 4 4 b \N \N +20250121171615893 20250121171615893_1_4 20250121171615893_7_1 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 10 10 b \N \N + +-- !refresh_all_partition_rewrite -- +20250121171615893 20250121171615893_0_0 20250121171615893_3_1 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 5 5 a \N \N +20250121171615893 20250121171615893_0_1 20250121171615893_5_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 7 7 a \N \N +20250121171615893 20250121171615893_0_2 20250121171615893_7_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 9 9 a \N \N +20250121171615893 20250121171615893_0_3 20250121171615893_0_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 1 1 a 1 2 +20250121171615893 20250121171615893_0_4 20250121171615893_2_0 par=a 58eabd3f-1996-4cb6-83e4-56fd11cb4e7d-0_0-30-108_20250121171615893.parquet 3 3 a \N \N +20250121171615893 20250121171615893_1_0 20250121171615893_4_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 6 6 b \N \N +20250121171615893 20250121171615893_1_1 20250121171615893_6_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 8 8 b \N \N +20250121171615893 20250121171615893_1_2 20250121171615893_1_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 2 2 b \N \N +20250121171615893 20250121171615893_1_3 20250121171615893_3_0 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 4 4 b \N \N +20250121171615893 20250121171615893_1_4 20250121171615893_7_1 par=b 7f98e9ac-bd11-48fd-ac80-9ca6dc1ddb34-0_1-30-109_20250121171615893.parquet 10 10 b \N \N + diff --git a/regression-test/data/external_table_p2/hudi/hudi_mtmv/test_hudi_rewrite_mtmv.out b/regression-test/data/external_table_p2/hudi/hudi_mtmv/test_hudi_rewrite_mtmv.out new file mode 100644 index 00000000000000..77597631587735 --- /dev/null +++ b/regression-test/data/external_table_p2/hudi/hudi_mtmv/test_hudi_rewrite_mtmv.out @@ -0,0 +1,16 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !refresh_one_partition -- +a 5 + +-- !refresh_one_partition_rewrite -- +a 5 +b 5 + +-- !refresh_auto -- +a 5 +b 5 + +-- !refresh_all_partition_rewrite -- +a 5 +b 5 + diff --git a/regression-test/data/mtmv_p0/test_hive_refresh_mtmv.out b/regression-test/data/mtmv_p0/test_hive_refresh_mtmv.out index d2536af5aa2fae..43fcbd3fba03e2 100644 --- a/regression-test/data/mtmv_p0/test_hive_refresh_mtmv.out +++ b/regression-test/data/mtmv_p0/test_hive_refresh_mtmv.out @@ -7,6 +7,9 @@ -- !mtmv_2 -- 1 1 2020 + +-- !mtmv_3 -- +1 1 2020 1 1 2020 -- !task_error -- @@ -23,6 +26,9 @@ SUCCESS -- !mtmv_2 -- 1 1 2020 + +-- !mtmv_3 -- +1 1 2020 1 1 2020 -- !task_error -- diff --git a/regression-test/data/mtmv_p0/test_hive_rewrite_mtmv.out b/regression-test/data/mtmv_p0/test_hive_rewrite_mtmv.out new file mode 100644 index 00000000000000..452cff71e53e9c --- /dev/null +++ b/regression-test/data/mtmv_p0/test_hive_rewrite_mtmv.out @@ -0,0 +1,31 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !refresh_one_partition -- +20230101 3 + +-- !refresh_one_partition_rewrite -- +20230101 3 +20230102 3 + +-- !refresh_complete -- +20230101 3 +20230102 3 + +-- !refresh_all_partition_rewrite -- +20230101 3 +20230102 3 + +-- !refresh_one_partition -- +20230101 3 + +-- !refresh_one_partition_rewrite -- +20230101 3 +20230102 3 + +-- !refresh_complete -- +20230101 3 +20230102 3 + +-- !refresh_all_partition_rewrite -- +20230101 3 +20230102 3 + diff --git a/regression-test/data/mtmv_p0/test_iceberg_mtmv.out b/regression-test/data/mtmv_p0/test_iceberg_mtmv.out index d0b47e27fffae5..47d2d3a61a2eb0 100644 --- a/regression-test/data/mtmv_p0/test_iceberg_mtmv.out +++ b/regression-test/data/mtmv_p0/test_iceberg_mtmv.out @@ -5,3 +5,146 @@ -- !mtmv -- 1000 +-- !test_ts_refresh1 -- +2024-10-26T01:02:03 1 +2024-10-27T01:02:03 2 +2024-10-27T21:02:03 3 + +-- !test_ts_refresh2 -- +2024-10-26T01:02:03 1 +2024-10-27T01:02:03 2 +2024-10-27T21:02:03 3 +2024-10-26T21:02:03 4 + +-- !test_ts_refresh3 -- +2024-10-26T01:02:03 1 +2024-10-27T01:02:03 2 +2024-10-27T21:02:03 3 +2024-10-26T21:02:03 4 +2024-10-26T01:22:03 5 + +-- !test_ts_refresh4 -- +2024-10-26T01:02:03 1 +2024-10-27T01:02:03 2 +2024-10-27T21:02:03 3 +2024-10-26T21:02:03 4 +2024-10-26T01:22:03 5 +2024-10-27T01:12:03 6 + +-- !test_ts_refresh5 -- +2024-10-26T01:02:03 1 +2024-10-27T01:02:03 2 +2024-10-27T21:02:03 3 +2024-10-26T21:02:03 4 +2024-10-26T01:22:03 5 +2024-10-27T01:12:03 6 + +-- !test_ts_refresh6 -- +2024-10-26T01:02:03 1 +2024-10-27T01:02:03 2 +2024-10-27T21:02:03 3 +2024-10-26T21:02:03 4 +2024-10-26T01:22:03 5 +2024-10-27T01:12:03 6 +2024-10-28T01:22:03 7 + +-- !test_ts_refresh_null -- +2024-10-26T01:02:03 1 +2024-10-27T01:02:03 2 +2024-10-27T21:02:03 3 +2024-10-26T21:02:03 4 +2024-10-26T01:22:03 5 +2024-10-27T01:12:03 6 +2024-10-28T01:22:03 7 +\N 8 + +-- !test_iceberg_table_partition_ts -- +ts_day=20022 types: [DATETIMEV2]; keys: [2024-10-26 00:00:00]; types: [DATETIMEV2]; keys: [2024-10-27 00:00:00]; +ts_day=20023 types: [DATETIMEV2]; keys: [2024-10-27 00:00:00]; types: [DATETIMEV2]; keys: [2024-10-28 00:00:00]; +ts_day=20024 types: [DATETIMEV2]; keys: [2024-10-28 00:00:00]; types: [DATETIMEV2]; keys: [2024-10-29 00:00:00]; +ts_day=null types: [DATETIMEV2]; keys: [0000-01-01 00:00:00]; types: [DATETIMEV2]; keys: [0000-01-01 00:00:01]; + +-- !unpartition -- +2024-10-26T01:02:03 1 +2024-10-27T01:02:03 2 +2024-10-27T21:02:03 3 +2024-10-26T21:02:03 4 +2024-10-26T01:22:03 5 +2024-10-27T01:12:03 6 +2024-10-28T01:22:03 7 +\N 8 + +-- !test_d_refresh1 -- +2024-08-26 1 +2024-09-17 2 +2024-09-27 3 + +-- !test_d_refresh2 -- +2024-08-26 1 +2024-09-17 2 +2024-09-27 3 +2024-09-01 4 + +-- !test_d_refresh3 -- +2024-08-26 1 +2024-09-17 2 +2024-09-27 3 +2024-09-01 4 +2024-08-22 5 + +-- !test_d_refresh4 -- +2024-08-26 1 +2024-09-17 2 +2024-09-27 3 +2024-09-01 4 +2024-08-22 5 +2024-09-30 6 + +-- !test_d_refresh5 -- +2024-08-26 1 +2024-09-17 2 +2024-09-27 3 +2024-09-01 4 +2024-08-22 5 +2024-09-30 6 +2024-10-28 7 + +-- !test_iceberg_table_partition_d -- +d_month=655 types: [DATEV2]; keys: [2024-08-01]; types: [DATEV2]; keys: [2024-09-01]; +d_month=656 types: [DATEV2]; keys: [2024-09-01]; types: [DATEV2]; keys: [2024-10-01]; +d_month=657 types: [DATEV2]; keys: [2024-10-01]; types: [DATEV2]; keys: [2024-11-01]; + +-- !refresh_one_partition -- +2024-01-01T00:00 4 + +-- !refresh_one_partition_rewrite -- +2024-01-01T00:00 4 +2024-01-02T00:00 3 + +-- !refresh_auto -- +2024-01-01T00:00 4 +2024-01-02T00:00 3 + +-- !refresh_all_partition_rewrite -- +2024-01-01T00:00 4 +2024-01-02T00:00 3 + +-- !evolution2 -- +ts_day=20060 types: [DATETIMEV2]; keys: [2024-12-03 00:00:00]; types: [DATETIMEV2]; keys: [2024-12-04 00:00:00]; +ts_month=657 types: [DATETIMEV2]; keys: [2024-10-01 00:00:00]; types: [DATETIMEV2]; keys: [2024-11-01 00:00:00]; +ts_month=658 types: [DATETIMEV2]; keys: [2024-11-01 00:00:00]; types: [DATETIMEV2]; keys: [2024-12-01 00:00:00]; + +-- !evolution3 -- +ts_day=20059 types: [DATETIMEV2]; keys: [2024-12-02 00:00:00]; types: [DATETIMEV2]; keys: [2024-12-03 00:00:00]; +ts_month=658 types: [DATETIMEV2]; keys: [2024-11-01 00:00:00]; types: [DATETIMEV2]; keys: [2024-12-01 00:00:00]; + +-- !evolution4 -- +ts_month=657 types: [DATETIMEV2]; keys: [2024-10-01 00:00:00]; types: [DATETIMEV2]; keys: [2024-11-01 00:00:00]; +ts_month=658 types: [DATETIMEV2]; keys: [2024-11-01 00:00:00]; types: [DATETIMEV2]; keys: [2024-12-01 00:00:00]; + +-- !evolution5 -- +ts_day=20078 types: [DATETIMEV2]; keys: [2024-12-21 00:00:00]; types: [DATETIMEV2]; keys: [2024-12-22 00:00:00]; +ts_hour=482139 types: [DATETIMEV2]; keys: [2025-01-01 03:00:00]; types: [DATETIMEV2]; keys: [2025-01-01 04:00:00]; +ts_month=657 types: [DATETIMEV2]; keys: [2024-10-01 00:00:00]; types: [DATETIMEV2]; keys: [2024-11-01 00:00:00]; +ts_month=658 types: [DATETIMEV2]; keys: [2024-11-01 00:00:00]; types: [DATETIMEV2]; keys: [2024-12-01 00:00:00]; + diff --git a/regression-test/data/mtmv_p0/test_paimon_mtmv.out b/regression-test/data/mtmv_p0/test_paimon_mtmv.out new file mode 100644 index 00000000000000..0cfe4bd293cf51 --- /dev/null +++ b/regression-test/data/mtmv_p0/test_paimon_mtmv.out @@ -0,0 +1,148 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !base_table -- +1 2 a +1 2 b +10 1 a +10 1 b +2 2 a +2 2 b +3 2 a +3 2 b +4 2 a +4 2 b +5 2 a +5 2 b +6 1 a +6 1 b +7 1 a +7 1 b +8 1 a +8 1 b +9 1 a +9 1 b + +-- !refresh_one_partition -- +1 2 a +10 1 a +2 2 a +3 2 a +4 2 a +5 2 a +6 1 a +7 1 a +8 1 a +9 1 a + +-- !refresh_auto -- +1 2 a +1 2 b +10 1 a +10 1 b +2 2 a +2 2 b +3 2 a +3 2 b +4 2 a +4 2 b +5 2 a +5 2 b +6 1 a +6 1 b +7 1 a +7 1 b +8 1 a +8 1 b +9 1 a +9 1 b + +-- !is_sync_before_rebuild -- +true + +-- !is_sync_after_rebuild -- +true + +-- !refresh_complete_rebuild -- +1 2 a +1 2 b +10 1 a +10 1 b +2 2 a +2 2 b +3 2 a +3 2 b +4 2 a +4 2 b +5 2 a +5 2 b +6 1 a +6 1 b +7 1 a +7 1 b +8 1 a +8 1 b +9 1 a +9 1 b + +-- !not_partition_before -- +false + +-- !not_partition -- +1 2 a +1 2 b +10 1 a +10 1 b +2 2 a +2 2 b +3 2 a +3 2 b +4 2 a +4 2 b +5 2 a +5 2 b +6 1 a +6 1 b +7 1 a +7 1 b +8 1 a +8 1 b +9 1 a +9 1 b + +-- !not_partition_after -- +true + +-- !join_one_partition -- +1 2 a 1 2 +10 1 a \N \N +2 2 a \N \N +3 2 a \N \N +4 2 a \N \N +5 2 a \N \N +6 1 a \N \N +7 1 a \N \N +8 1 a \N \N +9 1 a \N \N + +-- !two_partition -- +1 2020-01-01 bj +2 2020-01-01 sh +3 2038-01-01 bj +4 2038-01-01 sh +5 2038-01-02 bj + +-- !limit_partition -- +3 2038-01-01 bj +4 2038-01-01 sh +5 2038-01-02 bj + +-- !null_partition -- +1 bj +4 null +5 NULL + +-- !date_partition_base_table -- +1 2020-01-01 + +-- !date_partition -- +1 2020-01-01 + diff --git a/regression-test/data/mtmv_p0/test_paimon_olap_rewrite_mtmv.out b/regression-test/data/mtmv_p0/test_paimon_olap_rewrite_mtmv.out new file mode 100644 index 00000000000000..09d23b7736e1d8 --- /dev/null +++ b/regression-test/data/mtmv_p0/test_paimon_olap_rewrite_mtmv.out @@ -0,0 +1,79 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !refresh_one_partition -- +1 2 a 1 2 +10 1 a \N \N +2 2 a \N \N +3 2 a \N \N +4 2 a \N \N +5 2 a \N \N +6 1 a \N \N +7 1 a \N \N +8 1 a \N \N +9 1 a \N \N + +-- !refresh_one_partition_rewrite -- +1 2 a 1 2 +1 2 b 1 2 +10 1 a \N \N +10 1 b \N \N +2 2 a \N \N +2 2 b \N \N +3 2 a \N \N +3 2 b \N \N +4 2 a \N \N +4 2 b \N \N +5 2 a \N \N +5 2 b \N \N +6 1 a \N \N +6 1 b \N \N +7 1 a \N \N +7 1 b \N \N +8 1 a \N \N +8 1 b \N \N +9 1 a \N \N +9 1 b \N \N + +-- !refresh_auto -- +1 2 a 1 2 +1 2 b 1 2 +10 1 a \N \N +10 1 b \N \N +2 2 a \N \N +2 2 b \N \N +3 2 a \N \N +3 2 b \N \N +4 2 a \N \N +4 2 b \N \N +5 2 a \N \N +5 2 b \N \N +6 1 a \N \N +6 1 b \N \N +7 1 a \N \N +7 1 b \N \N +8 1 a \N \N +8 1 b \N \N +9 1 a \N \N +9 1 b \N \N + +-- !refresh_all_partition_rewrite -- +1 2 a 1 2 +1 2 b 1 2 +10 1 a \N \N +10 1 b \N \N +2 2 a \N \N +2 2 b \N \N +3 2 a \N \N +3 2 b \N \N +4 2 a \N \N +4 2 b \N \N +5 2 a \N \N +5 2 b \N \N +6 1 a \N \N +6 1 b \N \N +7 1 a \N \N +7 1 b \N \N +8 1 a \N \N +8 1 b \N \N +9 1 a \N \N +9 1 b \N \N + diff --git a/regression-test/data/mtmv_p0/test_paimon_rewrite_mtmv.out b/regression-test/data/mtmv_p0/test_paimon_rewrite_mtmv.out new file mode 100644 index 00000000000000..63bda82c1db5bd --- /dev/null +++ b/regression-test/data/mtmv_p0/test_paimon_rewrite_mtmv.out @@ -0,0 +1,16 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !refresh_one_partition -- +a 10 + +-- !refresh_one_partition_rewrite -- +a 10 +b 10 + +-- !refresh_auto -- +a 10 +b 10 + +-- !refresh_all_partition_rewrite -- +a 10 +b 10 + diff --git a/regression-test/suites/external_table_p2/hudi/hudi_mtmv/test_hudi_mtmv.groovy b/regression-test/suites/external_table_p2/hudi/hudi_mtmv/test_hudi_mtmv.groovy new file mode 100644 index 00000000000000..5bfcea11d67281 --- /dev/null +++ b/regression-test/suites/external_table_p2/hudi/hudi_mtmv/test_hudi_mtmv.groovy @@ -0,0 +1,256 @@ +// 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. + +suite("test_hudi_mtmv", "p2,external,hudi,external_remote,external_remote_hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHudiTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disabled hudi test") + return + } + String suiteName = "test_hudi_mtmv" + String catalogName = "${suiteName}_catalog" + String mvName = "${suiteName}_mv" + String dbName = context.config.getDbNameByFile(context.file) + String otherDbName = "${suiteName}_otherdb" + String tableName = "${suiteName}_table" + + sql """drop database if exists ${otherDbName}""" + sql """create database ${otherDbName}""" + sql """ + CREATE TABLE ${otherDbName}.${tableName} ( + `user_id` INT, + `num` INT + ) ENGINE=OLAP + DUPLICATE KEY(`user_id`) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1') ; + """ + + sql """ + insert into ${otherDbName}.${tableName} values(1,2); + """ + + String props = context.config.otherConfigs.get("hudiEmrCatalog") + + sql """drop catalog if exists ${catalogName}""" + sql """CREATE CATALOG if not exists ${catalogName} PROPERTIES ( + ${props} + );""" + + order_qt_base_table """ select * from ${catalogName}.hudi_mtmv_regression_test.hudi_table_1; """ + + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`par`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_1; + """ + def showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_a")) + assertTrue(showPartitionsResult.toString().contains("p_b")) + + // refresh one partitions + sql """ + REFRESH MATERIALIZED VIEW ${mvName} partitions(p_a); + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_one_partition "SELECT * FROM ${mvName} " + + //refresh auto + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_auto "SELECT * FROM ${mvName} " + order_qt_is_sync_before_rebuild "select SyncWithBaseTables from mv_infos('database'='${dbName}') where Name='${mvName}'" + + // rebuild catalog, should not Affects MTMV + sql """drop catalog if exists ${catalogName}""" + sql """CREATE CATALOG if not exists ${catalogName} PROPERTIES ( + ${props} + );""" + order_qt_is_sync_after_rebuild "select SyncWithBaseTables from mv_infos('database'='${dbName}') where Name='${mvName}'" + + // should refresh normal after catalog rebuild + sql """ + REFRESH MATERIALIZED VIEW ${mvName} complete + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_complete_rebuild "SELECT * FROM ${mvName} " + + sql """drop materialized view if exists ${mvName};""" + + // not have partition + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + KEY(`id`) + COMMENT "comment1" + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1',"grace_period"="333") + AS + SELECT id,age,par FROM ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_1; + """ + order_qt_not_partition_before "select SyncWithBaseTables from mv_infos('database'='${dbName}') where Name='${mvName}'" + //should can refresh auto + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_not_partition "SELECT * FROM ${mvName} " + order_qt_not_partition_after "select SyncWithBaseTables from mv_infos('database'='${dbName}') where Name='${mvName}'" + sql """drop materialized view if exists ${mvName};""" + + // refresh on schedule + // sql """ + // CREATE MATERIALIZED VIEW ${mvName} + // BUILD IMMEDIATE REFRESH COMPLETE ON SCHEDULE EVERY 10 SECOND STARTS "9999-12-13 21:07:09" + // KEY(`id`) + // COMMENT "comment1" + // DISTRIBUTED BY HASH(`id`) BUCKETS 2 + // PROPERTIES ('replication_num' = '1',"grace_period"="333") + // AS + // SELECT * FROM ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_1; + // """ + // waitingMTMVTaskFinishedByMvName(mvName) + // sql """drop materialized view if exists ${mvName};""" + + // refresh on schedule + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD IMMEDIATE REFRESH AUTO ON commit + KEY(`id`) + COMMENT "comment1" + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1',"grace_period"="333") + AS + SELECT id,age,par FROM ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_1; + """ + waitingMTMVTaskFinishedByMvName(mvName) + sql """drop materialized view if exists ${mvName};""" + + // cross db and join internal table + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`par`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_1 a left join internal.${otherDbName}.${tableName} b on a.id=b.user_id; + """ + def showJoinPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showJoinPartitionsResult: " + showJoinPartitionsResult.toString()) + assertTrue(showJoinPartitionsResult.toString().contains("p_a")) + assertTrue(showJoinPartitionsResult.toString().contains("p_b")) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} partitions(p_a); + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_join_one_partition "SELECT * FROM ${mvName} " + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`create_date`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_two_partitions; + """ + def showTwoPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showTwoPartitionsResult: " + showTwoPartitionsResult.toString()) + assertTrue(showTwoPartitionsResult.toString().contains("p_20200101")) + assertTrue(showTwoPartitionsResult.toString().contains("p_20380101")) + assertTrue(showTwoPartitionsResult.toString().contains("p_20380102")) + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto; + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_two_partition "SELECT * FROM ${mvName} " + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`create_date`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1','partition_sync_limit'='2','partition_date_format'='%Y-%m-%d', + 'partition_sync_time_unit'='MONTH') + AS + SELECT * FROM ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_two_partitions; + """ + def showLimitPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showLimitPartitionsResult: " + showLimitPartitionsResult.toString()) + assertFalse(showLimitPartitionsResult.toString().contains("p_20200101")) + assertTrue(showLimitPartitionsResult.toString().contains("p_20380101")) + assertTrue(showLimitPartitionsResult.toString().contains("p_20380102")) + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto; + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_limit_partition "SELECT * FROM ${mvName} " + sql """drop materialized view if exists ${mvName};""" + + // not allow date trunc + test { + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by (date_trunc(`create_date`,'month')) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1','partition_sync_limit'='2','partition_date_format'='%Y-%m-%d', + 'partition_sync_time_unit'='MONTH') + AS + SELECT * FROM ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_two_partitions; + """ + exception "only support" + } + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`region`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_null_partition; + """ + def showNullPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showNullPartitionsResult: " + showNullPartitionsResult.toString()) + // assertTrue(showNullPartitionsResult.toString().contains("p_null")) + assertTrue(showNullPartitionsResult.toString().contains("p_NULL")) + assertTrue(showNullPartitionsResult.toString().contains("p_bj")) + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto; + """ + waitingMTMVTaskFinishedByMvName(mvName) + // Will lose null data + order_qt_null_partition "SELECT * FROM ${mvName} " + sql """drop materialized view if exists ${mvName};""" + + sql """drop catalog if exists ${catalogName}""" + +} diff --git a/regression-test/suites/external_table_p2/hudi/hudi_mtmv/test_hudi_olap_rewrite_mtmv.groovy b/regression-test/suites/external_table_p2/hudi/hudi_mtmv/test_hudi_olap_rewrite_mtmv.groovy new file mode 100644 index 00000000000000..a0ac9b0783f3f1 --- /dev/null +++ b/regression-test/suites/external_table_p2/hudi/hudi_mtmv/test_hudi_olap_rewrite_mtmv.groovy @@ -0,0 +1,108 @@ +// 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. + +suite("test_hudi_olap_rewrite_mtmv", "p2,external,hudi,external_remote,external_remote_hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHudiTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disabled hudi test") + return + } + String suiteName = "test_hudi_olap_rewrite_mtmv" + String catalogName = "${suiteName}_catalog" + String mvName = "${suiteName}_mv" + String dbName = context.config.getDbNameByFile(context.file) + String tableName = "${suiteName}_table" + sql """drop table if exists ${tableName}""" + sql """ + CREATE TABLE ${tableName} ( + `user_id` INT, + `num` INT + ) ENGINE=OLAP + DUPLICATE KEY(`user_id`) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1') ; + """ + sql """ + insert into ${tableName} values(1,2); + """ + + sql """analyze table internal.`${dbName}`. ${tableName} with sync""" + sql """alter table internal.`${dbName}`. ${tableName} modify column user_id set stats ('row_count'='1');""" + + String props = context.config.otherConfigs.get("hudiEmrCatalog") + + sql """set materialized_view_rewrite_enable_contain_external_table=true;""" + String mvSql = "SELECT * FROM ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_1 a left join ${tableName} b on a.id=b.user_id;"; + + sql """drop catalog if exists ${catalogName}""" + sql """CREATE CATALOG if not exists ${catalogName} PROPERTIES ( + ${props} + );""" + + sql """analyze table ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_1 with sync""" + sql """alter table ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_1 modify column par set stats ('row_count'='10');""" + + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`par`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mvSql} + """ + def showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_a")) + assertTrue(showPartitionsResult.toString().contains("p_b")) + + // refresh one partitions + sql """ + REFRESH MATERIALIZED VIEW ${mvName} partitions(p_a); + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_one_partition "SELECT * FROM ${mvName} " + + def explainOnePartition = sql """ explain ${mvSql} """ + logger.info("explainOnePartition: " + explainOnePartition.toString()) + assertTrue(explainOnePartition.toString().contains("VUNION")) + order_qt_refresh_one_partition_rewrite "${mvSql}" + + mv_rewrite_success("${mvSql}", "${mvName}") + + // select p_b should not rewrite + mv_rewrite_fail("SELECT * FROM ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_1 a left join ${tableName} b on a.id=b.user_id where a.par='b';", "${mvName}") + + //refresh auto + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_auto "SELECT * FROM ${mvName} " + + def explainAllPartition = sql """ explain ${mvSql}; """ + logger.info("explainAllPartition: " + explainAllPartition.toString()) + assertTrue(explainAllPartition.toString().contains("VOlapScanNode")) + order_qt_refresh_all_partition_rewrite "${mvSql}" + + mv_rewrite_success("${mvSql}", "${mvName}") + + sql """drop materialized view if exists ${mvName};""" + sql """drop catalog if exists ${catalogName}""" +} diff --git a/regression-test/suites/external_table_p2/hudi/hudi_mtmv/test_hudi_rewrite_mtmv.groovy b/regression-test/suites/external_table_p2/hudi/hudi_mtmv/test_hudi_rewrite_mtmv.groovy new file mode 100644 index 00000000000000..95c71c484759ce --- /dev/null +++ b/regression-test/suites/external_table_p2/hudi/hudi_mtmv/test_hudi_rewrite_mtmv.groovy @@ -0,0 +1,91 @@ +// 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. + +suite("test_hudi_rewrite_mtmv", "p2,external,hudi,external_remote,external_remote_hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHudiTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disabled hudi test") + return + } + String suiteName = "test_hudi_rewrite_mtmv" + String catalogName = "${suiteName}_catalog" + String mvName = "${suiteName}_mv" + String dbName = context.config.getDbNameByFile(context.file) + + String props = context.config.otherConfigs.get("hudiEmrCatalog") + + sql """set materialized_view_rewrite_enable_contain_external_table=true;""" + String mvSql = "SELECT par,count(*) as num FROM ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_1 group by par;"; + + sql """drop catalog if exists ${catalogName}""" + sql """CREATE CATALOG if not exists ${catalogName} PROPERTIES ( + ${props} + );""" + + sql """analyze table ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_1 with sync""" + sql """alter table ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_1 modify column par set stats ('row_count'='10');""" + + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`par`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mvSql} + """ + def showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_a")) + assertTrue(showPartitionsResult.toString().contains("p_b")) + + // refresh one partitions + sql """ + REFRESH MATERIALIZED VIEW ${mvName} partitions(p_a); + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_one_partition "SELECT * FROM ${mvName} " + + def explainOnePartition = sql """ explain ${mvSql} """ + logger.info("explainOnePartition: " + explainOnePartition.toString()) + assertTrue(explainOnePartition.toString().contains("VUNION")) + order_qt_refresh_one_partition_rewrite "${mvSql}" + + mv_rewrite_success("${mvSql}", "${mvName}") + + // select p_b should not rewrite + mv_rewrite_fail("SELECT par,count(*) as num FROM ${catalogName}.`hudi_mtmv_regression_test`.hudi_table_1 where par='b' group by par;", "${mvName}") + + //refresh auto + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_auto "SELECT * FROM ${mvName} " + + def explainAllPartition = sql """ explain ${mvSql}; """ + logger.info("explainAllPartition: " + explainAllPartition.toString()) + assertTrue(explainAllPartition.toString().contains("VOlapScanNode")) + order_qt_refresh_all_partition_rewrite "${mvSql}" + + mv_rewrite_success("${mvSql}", "${mvName}") + + sql """drop materialized view if exists ${mvName};""" + sql """drop catalog if exists ${catalogName}""" +} diff --git a/regression-test/suites/mtmv_p0/test_hive_refresh_mtmv.groovy b/regression-test/suites/mtmv_p0/test_hive_refresh_mtmv.groovy index f2cfae8c3e92ba..f2e31d96a1ea3e 100644 --- a/regression-test/suites/mtmv_p0/test_hive_refresh_mtmv.groovy +++ b/regression-test/suites/mtmv_p0/test_hive_refresh_mtmv.groovy @@ -103,6 +103,14 @@ suite("test_hive_refresh_mtmv", "p0,external,hive,external_docker,external_docke // hive data change logger.info("hive sql: " + insert_str) + + // If we didn't refresh the catalog, the data in mv should still be old + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto + """ + waitingMTMVTaskFinished(jobName) + order_qt_mtmv_2 "SELECT * FROM ${mvName} order by user_id" + hive_docker """ ${insert_str} """ sql """ REFRESH catalog ${catalog_name} @@ -111,7 +119,7 @@ suite("test_hive_refresh_mtmv", "p0,external,hive,external_docker,external_docke REFRESH MATERIALIZED VIEW ${mvName} auto """ waitingMTMVTaskFinished(jobName) - order_qt_mtmv_2 "SELECT * FROM ${mvName} order by user_id" + order_qt_mtmv_3 "SELECT * FROM ${mvName} order by user_id" // hive add partition def add_partition2021_str = """ diff --git a/regression-test/suites/mtmv_p0/test_hive_rewrite_mtmv.groovy b/regression-test/suites/mtmv_p0/test_hive_rewrite_mtmv.groovy new file mode 100644 index 00000000000000..f10d6bd65b4d3b --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_hive_rewrite_mtmv.groovy @@ -0,0 +1,89 @@ +// 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. + +suite("test_hive_rewrite_mtmv", "p0,external,hive,external_docker,external_docker_hive") { + String enabled = context.config.otherConfigs.get("enableHiveTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("diable Hive test.") + return; + } + String suiteName = "test_hive_rewrite_mtmv" + String catalogName = "${suiteName}_catalog" + String mvName = "${suiteName}_mv" + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + sql """set materialized_view_rewrite_enable_contain_external_table=true;""" + String mvSql = "SELECT part_col,count(*) as num FROM ${catalogName}.`default`.mtmv_base1 group by part_col;"; + for (String hivePrefix : ["hive2", "hive3"]) { + String hms_port = context.config.otherConfigs.get(hivePrefix + "HmsPort") + sql """drop catalog if exists ${catalogName}""" + sql """create catalog if not exists ${catalogName} properties ( + "type"="hms", + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}' + );""" + sql """analyze table ${catalogName}.`default`.mtmv_base1 with sync""" + sql """alter table ${catalogName}.`default`.mtmv_base1 modify column part_col set stats ('row_count'='6');""" + + sql """drop materialized view if exists ${mvName};""" + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`part_col`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mvSql} + """ + def showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_20230101")) + assertTrue(showPartitionsResult.toString().contains("p_20230102")) + + // refresh one partitions + sql """ + REFRESH MATERIALIZED VIEW ${mvName} partitions(p_20230101); + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_one_partition "SELECT * FROM ${mvName}" + + def explainOnePartition = sql """ explain ${mvSql} """ + logger.info("explainOnePartition: " + explainOnePartition.toString()) + assertTrue(explainOnePartition.toString().contains("VUNION")) + assertTrue(explainOnePartition.toString().contains("part_col[#4] = 20230102")) + order_qt_refresh_one_partition_rewrite "${mvSql}" + + mv_rewrite_success("${mvSql}", "${mvName}") + + //refresh complete + sql """ + REFRESH MATERIALIZED VIEW ${mvName} complete + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_complete "SELECT * FROM ${mvName}" + + def explainAllPartition = sql """ explain ${mvSql}; """ + logger.info("explainAllPartition: " + explainAllPartition.toString()) + assertTrue(explainAllPartition.toString().contains("VOlapScanNode")) + assertTrue(explainAllPartition.toString().contains("partitions=2/2")) + order_qt_refresh_all_partition_rewrite "${mvSql}" + + mv_rewrite_success("${mvSql}", "${mvName}") + + sql """drop materialized view if exists ${mvName};""" + sql """drop catalog if exists ${catalogName}""" + } +} + diff --git a/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy b/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy index 3004f1677bded6..30b9e8e9f7329c 100644 --- a/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy +++ b/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy @@ -65,5 +65,255 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_ sql """drop materialized view if exists ${mvName};""" sql """ drop catalog if exists ${catalog_name} """ } + + // Test partition refresh. + // Use hms catalog to avoid rest catalog fail to write caused by sqlite database file locked. + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String catalog_name = "iceberg_mtmv_catalog_hms"; + String mvUnpartition = "test_iceberg_unpartition" + String mvName1 = "test_iceberg_mtmv_ts" + String mvName2 = "test_iceberg_mtmv_d" + String dbName = "regression_test_mtmv_partition_p0" + String icebergDb = "iceberg_mtmv_partition" + String icebergTable1 = "tstable" + String icebergTable2 = "dtable" + String icebergTable3 = "union_test" + sql """drop catalog if exists ${catalog_name} """ + sql """CREATE CATALOG ${catalog_name} PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='rest', + 'uri' = 'http://${externalEnvIp}:${rest_port}', + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", + "s3.region" = "us-east-1" + );""" + + sql """switch internal""" + sql """drop database if exists ${dbName}""" + sql """create database if not exists ${dbName}""" + sql """use ${dbName}""" + + sql """drop table if exists ${catalog_name}.${icebergDb}.${icebergTable1}""" + sql """drop table if exists ${catalog_name}.${icebergDb}.${icebergTable2}""" + sql """create database if not exists ${catalog_name}.${icebergDb}""" + sql """ + CREATE TABLE ${catalog_name}.${icebergDb}.${icebergTable1} ( + ts DATETIME, + value INT) + ENGINE=iceberg + PARTITION BY LIST (DAY(ts)) (); + """ + sql """insert into ${catalog_name}.${icebergDb}.${icebergTable1} values ('2024-10-26 01:02:03', 1), ('2024-10-27 01:02:03', 2), ('2024-10-27 21:02:03', 3)""" + sql """CREATE MATERIALIZED VIEW ${mvName1} BUILD DEFERRED REFRESH AUTO ON MANUAL partition by(`ts`) DISTRIBUTED BY RANDOM BUCKETS 2 PROPERTIES ('replication_num' = '1') as SELECT * FROM ${catalog_name}.${icebergDb}.${icebergTable1}""" + sql """REFRESH MATERIALIZED VIEW ${mvName1} complete""" + waitingMTMVTaskFinishedByMvName(mvName1, dbName) + qt_test_ts_refresh1 "select * from ${mvName1} order by value" + + sql """insert into ${catalog_name}.${icebergDb}.${icebergTable1} values ('2024-10-26 21:02:03', 4)""" + sql """REFRESH MATERIALIZED VIEW ${mvName1} auto""" + waitingMTMVTaskFinishedByMvName(mvName1, dbName) + qt_test_ts_refresh2 """select * from ${mvName1} order by value""" + + sql """insert into ${catalog_name}.${icebergDb}.${icebergTable1} values ('2024-10-26 01:22:03', 5), ('2024-10-27 01:12:03', 6);""" + sql """REFRESH MATERIALIZED VIEW ${mvName1} partitions(p_20241026000000_20241027000000);""" + waitingMTMVTaskFinishedByMvName(mvName1, dbName) + qt_test_ts_refresh3 """select * from ${mvName1} order by value""" + + sql """REFRESH MATERIALIZED VIEW ${mvName1} auto""" + waitingMTMVTaskFinishedByMvName(mvName1, dbName) + qt_test_ts_refresh4 """select * from ${mvName1} order by value""" + + sql """insert into ${catalog_name}.${icebergDb}.${icebergTable1} values ('2024-10-28 01:22:03', 7);""" + sql """REFRESH MATERIALIZED VIEW ${mvName1} partitions(p_20241026000000_20241027000000);""" + waitingMTMVTaskFinishedByMvName(mvName1, dbName) + qt_test_ts_refresh5 """select * from ${mvName1} order by value""" + + sql """REFRESH MATERIALIZED VIEW ${mvName1} auto""" + waitingMTMVTaskFinishedByMvName(mvName1, dbName) + qt_test_ts_refresh6 """select * from ${mvName1} order by value""" + + sql """insert into ${catalog_name}.${icebergDb}.${icebergTable1} values (null, 8);""" + sql """REFRESH MATERIALIZED VIEW ${mvName1} auto""" + waitingMTMVTaskFinishedByMvName(mvName1, dbName) + qt_test_ts_refresh_null """select * from ${mvName1} order by value""" + + qt_test_iceberg_table_partition_ts """show partitions from ${catalog_name}.${icebergDb}.${icebergTable1};""" + + def showPartitionsResult = sql """show partitions from ${mvName1}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_20241026000000_20241027000000")) + assertTrue(showPartitionsResult.toString().contains("p_20241027000000_20241028000000")) + assertTrue(showPartitionsResult.toString().contains("p_20241028000000_20241029000000")) + + sql """drop materialized view if exists ${mvName1};""" + + // Test unpartitioned mtmv + sql """ + CREATE MATERIALIZED VIEW ${mvUnpartition} + BUILD DEFERRED REFRESH AUTO ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${catalog_name}.${icebergDb}.${icebergTable1}; + """ + sql """ + REFRESH MATERIALIZED VIEW ${mvUnpartition} complete + """ + def jobName = getJobName(dbName, mvUnpartition); + waitingMTMVTaskFinished(jobName) + qt_unpartition "SELECT * FROM ${mvUnpartition} order by value" + + sql """drop materialized view if exists ${mvUnpartition};""" + sql """drop table if exists ${catalog_name}.${icebergDb}.${icebergTable1}""" + + + sql """ + CREATE TABLE ${catalog_name}.${icebergDb}.${icebergTable2} ( + d DATE, + value INT) + ENGINE=iceberg + PARTITION BY LIST (MONTH(d)) (); + """ + sql """insert into ${catalog_name}.${icebergDb}.${icebergTable2} values ('2024-08-26', 1), ('2024-09-17', 2), ('2024-09-27', 3);""" + sql """CREATE MATERIALIZED VIEW ${mvName2} BUILD DEFERRED REFRESH AUTO ON MANUAL partition by(`d`) DISTRIBUTED BY RANDOM BUCKETS 2 PROPERTIES ('replication_num' = '1') as SELECT * FROM ${catalog_name}.${icebergDb}.${icebergTable2}""" + sql """REFRESH MATERIALIZED VIEW ${mvName2} complete""" + waitingMTMVTaskFinishedByMvName(mvName2, dbName) + qt_test_d_refresh1 "select * from ${mvName2} order by value" + + sql """insert into ${catalog_name}.${icebergDb}.${icebergTable2} values ('2024-09-01', 4);""" + sql """REFRESH MATERIALIZED VIEW ${mvName2} auto""" + waitingMTMVTaskFinishedByMvName(mvName2, dbName) + qt_test_d_refresh2 "select * from ${mvName2} order by value" + + sql """insert into ${catalog_name}.${icebergDb}.${icebergTable2} values ('2024-08-22', 5), ('2024-09-30', 6);""" + sql """REFRESH MATERIALIZED VIEW ${mvName2} partitions(p_20240801_20240901);""" + waitingMTMVTaskFinishedByMvName(mvName2, dbName) + qt_test_d_refresh3 "select * from ${mvName2} order by value" + sql """REFRESH MATERIALIZED VIEW ${mvName2} partitions(p_20240901_20241001);""" + waitingMTMVTaskFinishedByMvName(mvName2, dbName) + qt_test_d_refresh4 "select * from ${mvName2} order by value" + + sql """insert into ${catalog_name}.${icebergDb}.${icebergTable2} values ('2024-10-28', 7);""" + sql """REFRESH MATERIALIZED VIEW ${mvName2} auto""" + waitingMTMVTaskFinishedByMvName(mvName2, dbName) + qt_test_d_refresh5 "select * from ${mvName2} order by value" + + qt_test_iceberg_table_partition_d """show partitions from ${catalog_name}.${icebergDb}.${icebergTable2};""" + + showPartitionsResult = sql """show partitions from ${mvName2}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_20240801_20240901")) + assertTrue(showPartitionsResult.toString().contains("p_20240901_20241001")) + assertTrue(showPartitionsResult.toString().contains("p_20241001_20241101")) + + sql """drop materialized view if exists ${mvName2};""" + sql """drop table if exists ${catalog_name}.${icebergDb}.${icebergTable2}""" + + // Test rewrite and union partitions + sql """set materialized_view_rewrite_enable_contain_external_table=true;""" + String mvSql = "SELECT par,count(*) as num FROM ${catalog_name}.${icebergDb}.${icebergTable3} group by par" + String mvName = "union_mv" + sql """drop table if exists ${catalog_name}.${icebergDb}.${icebergTable3}""" + sql """ + CREATE TABLE ${catalog_name}.${icebergDb}.${icebergTable3} ( + id int, + value int, + par datetime + ) ENGINE=iceberg + PARTITION BY LIST (day(par)) (); + """ + sql """insert into ${catalog_name}.${icebergDb}.${icebergTable3} values (1, 1, "2024-01-01"), (2, 1, "2024-01-01"), (3, 1, "2024-01-01"), (4, 1, "2024-01-01")""" + sql """insert into ${catalog_name}.${icebergDb}.${icebergTable3} values (1, 2, "2024-01-02"), (2, 2, "2024-01-02"), (3, 2, "2024-01-02")""" + sql """analyze table ${catalog_name}.${icebergDb}.${icebergTable3} with sync""" + + sql """drop materialized view if exists ${mvName};""" + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`par`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS ${mvSql} + """ + + def showPartitions = sql """show partitions from ${mvName}""" + logger.info("showPartitions: " + showPartitions.toString()) + assertTrue(showPartitions.toString().contains("p_20240101000000_20240102000000")) + assertTrue(showPartitions.toString().contains("p_20240102000000_20240103000000")) + + // refresh one partiton + sql """REFRESH MATERIALIZED VIEW ${mvName} partitions(p_20240101000000_20240102000000);""" + waitingMTMVTaskFinishedByMvName(mvName, dbName) + order_qt_refresh_one_partition "SELECT * FROM ${mvName} " + def explainOnePartition = sql """ explain ${mvSql} """ + logger.info("explainOnePartition: " + explainOnePartition.toString()) + assertTrue(explainOnePartition.toString().contains("VUNION")) + order_qt_refresh_one_partition_rewrite "${mvSql}" + mv_rewrite_success("${mvSql}", "${mvName}") + + //refresh auto + sql """REFRESH MATERIALIZED VIEW ${mvName} auto""" + waitingMTMVTaskFinishedByMvName(mvName, dbName) + order_qt_refresh_auto "SELECT * FROM ${mvName} " + def explainAllPartition = sql """ explain ${mvSql}; """ + logger.info("explainAllPartition: " + explainAllPartition.toString()) + assertTrue(explainAllPartition.toString().contains("VOlapScanNode")) + order_qt_refresh_all_partition_rewrite "${mvSql}" + mv_rewrite_success("${mvSql}", "${mvName}") + + sql """drop materialized view if exists ${mvName};""" + sql """drop table if exists ${catalog_name}.${icebergDb}.${icebergTable3}""" + + sql """use ${catalog_name}.test_db""" + qt_evolution2 "show partitions from replace_partition2" + qt_evolution3 "show partitions from replace_partition3" + qt_evolution4 "show partitions from replace_partition4" + qt_evolution5 "show partitions from replace_partition5" + + test { + sql "show partitions from replace_partition1" + // check exception message contains + exception "is not a supported partition table" + } + + test { + sql "show partitions from no_partition" + // check exception message contains + exception "is not a supported partition table" + } + + test { + sql "show partitions from not_support_trans" + // check exception message contains + exception "is not a supported partition table" + } + + test { + sql "show partitions from drop_partition1" + // check exception message contains + exception "is not a supported partition table" + } + + test { + sql "show partitions from drop_partition2" + // check exception message contains + exception "is not a supported partition table" + } + + test { + sql "show partitions from add_partition1" + // check exception message contains + exception "is not a supported partition table" + } + + test { + sql "show partitions from add_partition2" + // check exception message contains + exception "is not a supported partition table" + } + + sql """ drop catalog if exists ${catalog_name} """ + } } diff --git a/regression-test/suites/mtmv_p0/test_paimon_mtmv.groovy b/regression-test/suites/mtmv_p0/test_paimon_mtmv.groovy new file mode 100644 index 00000000000000..98477ab92d542e --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_paimon_mtmv.groovy @@ -0,0 +1,300 @@ +// 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. + +suite("test_paimon_mtmv", "p0,external,mtmv,external_docker,external_docker_doris") { + String enabled = context.config.otherConfigs.get("enablePaimonTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disabled paimon test") + return + } + String suiteName = "test_paimon_mtmv" + String catalogName = "${suiteName}_catalog" + String mvName = "${suiteName}_mv" + String dbName = context.config.getDbNameByFile(context.file) + String otherDbName = "${suiteName}_otherdb" + String tableName = "${suiteName}_table" + + sql """drop database if exists ${otherDbName}""" + sql """create database ${otherDbName}""" + sql """ + CREATE TABLE ${otherDbName}.${tableName} ( + `user_id` INT, + `num` INT + ) ENGINE=OLAP + DUPLICATE KEY(`user_id`) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1') ; + """ + + sql """ + insert into ${otherDbName}.${tableName} values(1,2); + """ + + String minio_port = context.config.otherConfigs.get("iceberg_minio_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + + sql """drop catalog if exists ${catalogName}""" + sql """CREATE CATALOG ${catalogName} PROPERTIES ( + 'type'='paimon', + 'warehouse' = 's3://warehouse/wh/', + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", + "s3.region" = "us-east-1" + );""" + + order_qt_base_table """ select * from ${catalogName}.test_paimon_spark.test_tb_mix_format ; """ + + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`par`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${catalogName}.`test_paimon_spark`.test_tb_mix_format; + """ + def showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_a")) + assertTrue(showPartitionsResult.toString().contains("p_b")) + + // refresh one partitions + sql """ + REFRESH MATERIALIZED VIEW ${mvName} partitions(p_a); + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_one_partition "SELECT * FROM ${mvName} " + + //refresh auto + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_auto "SELECT * FROM ${mvName} " + order_qt_is_sync_before_rebuild "select SyncWithBaseTables from mv_infos('database'='${dbName}') where Name='${mvName}'" + + // rebuild catalog, should not Affects MTMV + sql """drop catalog if exists ${catalogName}""" + sql """ + CREATE CATALOG ${catalogName} PROPERTIES ( + 'type'='paimon', + 'warehouse' = 's3://warehouse/wh/', + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", + "s3.region" = "us-east-1" + ); + """ + order_qt_is_sync_after_rebuild "select SyncWithBaseTables from mv_infos('database'='${dbName}') where Name='${mvName}'" + + // should refresh normal after catalog rebuild + sql """ + REFRESH MATERIALIZED VIEW ${mvName} complete + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_complete_rebuild "SELECT * FROM ${mvName} " + + sql """drop materialized view if exists ${mvName};""" + + // not have partition + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + KEY(`id`) + COMMENT "comment1" + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1',"grace_period"="333") + AS + SELECT * FROM ${catalogName}.`test_paimon_spark`.test_tb_mix_format; + """ + order_qt_not_partition_before "select SyncWithBaseTables from mv_infos('database'='${dbName}') where Name='${mvName}'" + //should can refresh auto + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_not_partition "SELECT * FROM ${mvName} " + order_qt_not_partition_after "select SyncWithBaseTables from mv_infos('database'='${dbName}') where Name='${mvName}'" + sql """drop materialized view if exists ${mvName};""" + + // refresh on schedule + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD IMMEDIATE REFRESH COMPLETE ON SCHEDULE EVERY 10 SECOND STARTS "9999-12-13 21:07:09" + KEY(`id`) + COMMENT "comment1" + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1',"grace_period"="333") + AS + SELECT * FROM ${catalogName}.`test_paimon_spark`.test_tb_mix_format; + """ + waitingMTMVTaskFinishedByMvName(mvName) + sql """drop materialized view if exists ${mvName};""" + + // refresh on schedule + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD IMMEDIATE REFRESH AUTO ON commit + KEY(`id`) + COMMENT "comment1" + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1',"grace_period"="333") + AS + SELECT * FROM ${catalogName}.`test_paimon_spark`.test_tb_mix_format; + """ + waitingMTMVTaskFinishedByMvName(mvName) + sql """drop materialized view if exists ${mvName};""" + + // cross db and join internal table + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`par`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${catalogName}.`test_paimon_spark`.test_tb_mix_format a left join internal.${otherDbName}.${tableName} b on a.id=b.user_id; + """ + def showJoinPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showJoinPartitionsResult: " + showJoinPartitionsResult.toString()) + assertTrue(showJoinPartitionsResult.toString().contains("p_a")) + assertTrue(showJoinPartitionsResult.toString().contains("p_b")) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} partitions(p_a); + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_join_one_partition "SELECT * FROM ${mvName} " + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`create_date`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${catalogName}.`test_paimon_spark`.two_partition; + """ + def showTwoPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showTwoPartitionsResult: " + showTwoPartitionsResult.toString()) + assertTrue(showTwoPartitionsResult.toString().contains("p_20200101")) + assertTrue(showTwoPartitionsResult.toString().contains("p_20380101")) + assertTrue(showTwoPartitionsResult.toString().contains("p_20380102")) + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto; + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_two_partition "SELECT * FROM ${mvName} " + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`create_date`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1','partition_sync_limit'='2','partition_date_format'='%Y-%m-%d', + 'partition_sync_time_unit'='MONTH') + AS + SELECT * FROM ${catalogName}.`test_paimon_spark`.two_partition; + """ + def showLimitPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showLimitPartitionsResult: " + showLimitPartitionsResult.toString()) + assertFalse(showLimitPartitionsResult.toString().contains("p_20200101")) + assertTrue(showLimitPartitionsResult.toString().contains("p_20380101")) + assertTrue(showLimitPartitionsResult.toString().contains("p_20380102")) + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto; + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_limit_partition "SELECT * FROM ${mvName} " + sql """drop materialized view if exists ${mvName};""" + + // not allow date trunc + test { + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by (date_trunc(`create_date`,'month')) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1','partition_sync_limit'='2','partition_date_format'='%Y-%m-%d', + 'partition_sync_time_unit'='MONTH') + AS + SELECT * FROM ${catalogName}.`test_paimon_spark`.two_partition; + """ + exception "only support" + } + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`region`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${catalogName}.`test_paimon_spark`.null_partition; + """ + def showNullPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showNullPartitionsResult: " + showNullPartitionsResult.toString()) + assertTrue(showNullPartitionsResult.toString().contains("p_null")) + assertTrue(showNullPartitionsResult.toString().contains("p_NULL")) + assertTrue(showNullPartitionsResult.toString().contains("p_bj")) + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto; + """ + waitingMTMVTaskFinishedByMvName(mvName) + // Will lose null data + order_qt_null_partition "SELECT * FROM ${mvName} " + sql """drop materialized view if exists ${mvName};""" + + // date type will has problem + order_qt_date_partition_base_table "SELECT * FROM ${catalogName}.`test_paimon_spark`.date_partition" + test { + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by (`create_date`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${catalogName}.`test_paimon_spark`.date_partition; + """ + exception "Unable to find a suitable base table" + } + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${catalogName}.`test_paimon_spark`.date_partition; + """ + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto; + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_date_partition "SELECT * FROM ${mvName} " + + sql """drop materialized view if exists ${mvName};""" + sql """drop catalog if exists ${catalogName}""" + +} + diff --git a/regression-test/suites/mtmv_p0/test_paimon_olap_rewrite_mtmv.groovy b/regression-test/suites/mtmv_p0/test_paimon_olap_rewrite_mtmv.groovy new file mode 100644 index 00000000000000..a3ac1c048d30da --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_paimon_olap_rewrite_mtmv.groovy @@ -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. + +suite("test_paimon_olap_rewrite_mtmv", "p0,external,mtmv,external_docker,external_docker_doris") { + String enabled = context.config.otherConfigs.get("enablePaimonTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disabled paimon test") + return + } + String suiteName = "test_paimon_olap_rewrite_mtmv" + String catalogName = "${suiteName}_catalog" + String mvName = "${suiteName}_mv" + String dbName = context.config.getDbNameByFile(context.file) + String tableName = "${suiteName}_table" + sql """drop table if exists ${tableName}""" + sql """ + CREATE TABLE ${tableName} ( + `user_id` INT, + `num` INT + ) ENGINE=OLAP + DUPLICATE KEY(`user_id`) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1') ; + """ + sql """ + insert into ${tableName} values(1,2); + """ + + sql """analyze table internal.`${dbName}`. ${tableName} with sync""" + sql """alter table internal.`${dbName}`. ${tableName} modify column user_id set stats ('row_count'='1');""" + + String minio_port = context.config.otherConfigs.get("iceberg_minio_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + + sql """set materialized_view_rewrite_enable_contain_external_table=true;""" + String mvSql = "SELECT * FROM ${catalogName}.`test_paimon_spark`.test_tb_mix_format a left join ${tableName} b on a.id=b.user_id;"; + + sql """drop catalog if exists ${catalogName}""" + sql """CREATE CATALOG ${catalogName} PROPERTIES ( + 'type'='paimon', + 'warehouse' = 's3://warehouse/wh/', + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", + "s3.region" = "us-east-1" + );""" + + sql """analyze table ${catalogName}.`test_paimon_spark`.test_tb_mix_format with sync""" + sql """alter table ${catalogName}.`test_paimon_spark`.test_tb_mix_format modify column par set stats ('row_count'='20');""" + + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`par`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mvSql} + """ + def showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_a")) + assertTrue(showPartitionsResult.toString().contains("p_b")) + + // refresh one partitions + sql """ + REFRESH MATERIALIZED VIEW ${mvName} partitions(p_a); + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_one_partition "SELECT * FROM ${mvName} " + + def explainOnePartition = sql """ explain ${mvSql} """ + logger.info("explainOnePartition: " + explainOnePartition.toString()) + assertTrue(explainOnePartition.toString().contains("VUNION")) + order_qt_refresh_one_partition_rewrite "${mvSql}" + + mv_rewrite_success("${mvSql}", "${mvName}") + + // select p_b should not rewrite + mv_rewrite_fail("SELECT * FROM ${catalogName}.`test_paimon_spark`.test_tb_mix_format a left join ${tableName} b on a.id=b.user_id where a.par='b';", "${mvName}") + + //refresh auto + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_auto "SELECT * FROM ${mvName} " + + def explainAllPartition = sql """ explain ${mvSql}; """ + logger.info("explainAllPartition: " + explainAllPartition.toString()) + assertTrue(explainAllPartition.toString().contains("VOlapScanNode")) + order_qt_refresh_all_partition_rewrite "${mvSql}" + + mv_rewrite_success("${mvSql}", "${mvName}") + + sql """drop materialized view if exists ${mvName};""" + sql """drop catalog if exists ${catalogName}""" +} + diff --git a/regression-test/suites/mtmv_p0/test_paimon_rewrite_mtmv.groovy b/regression-test/suites/mtmv_p0/test_paimon_rewrite_mtmv.groovy new file mode 100644 index 00000000000000..22a94d46635169 --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_paimon_rewrite_mtmv.groovy @@ -0,0 +1,98 @@ +// 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. + +suite("test_paimon_rewrite_mtmv", "p0,external,mtmv,external_docker,external_docker_doris") { + String enabled = context.config.otherConfigs.get("enablePaimonTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disabled paimon test") + return + } + String suiteName = "test_paimon_rewrite_mtmv" + String catalogName = "${suiteName}_catalog" + String mvName = "${suiteName}_mv" + String dbName = context.config.getDbNameByFile(context.file) + + String minio_port = context.config.otherConfigs.get("iceberg_minio_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + + sql """set materialized_view_rewrite_enable_contain_external_table=true;""" + String mvSql = "SELECT par,count(*) as num FROM ${catalogName}.`test_paimon_spark`.test_tb_mix_format group by par;"; + + sql """drop catalog if exists ${catalogName}""" + sql """CREATE CATALOG ${catalogName} PROPERTIES ( + 'type'='paimon', + 'warehouse' = 's3://warehouse/wh/', + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", + "s3.region" = "us-east-1" + );""" + + sql """analyze table ${catalogName}.`test_paimon_spark`.test_tb_mix_format with sync""" + sql """alter table ${catalogName}.`test_paimon_spark`.test_tb_mix_format modify column par set stats ('row_count'='20');""" + + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`par`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mvSql} + """ + def showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_a")) + assertTrue(showPartitionsResult.toString().contains("p_b")) + + // refresh one partitions + sql """ + REFRESH MATERIALIZED VIEW ${mvName} partitions(p_a); + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_one_partition "SELECT * FROM ${mvName} " + + def explainOnePartition = sql """ explain ${mvSql} """ + logger.info("explainOnePartition: " + explainOnePartition.toString()) + assertTrue(explainOnePartition.toString().contains("VUNION")) + order_qt_refresh_one_partition_rewrite "${mvSql}" + + mv_rewrite_success("${mvSql}", "${mvName}") + + // select p_b should not rewrite + mv_rewrite_fail("SELECT par,count(*) as num FROM ${catalogName}.`test_paimon_spark`.test_tb_mix_format where par='b' group by par;", "${mvName}") + + //refresh auto + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_auto "SELECT * FROM ${mvName} " + + def explainAllPartition = sql """ explain ${mvSql}; """ + logger.info("explainAllPartition: " + explainAllPartition.toString()) + assertTrue(explainAllPartition.toString().contains("VOlapScanNode")) + order_qt_refresh_all_partition_rewrite "${mvSql}" + + mv_rewrite_success("${mvSql}", "${mvName}") + + sql """drop materialized view if exists ${mvName};""" + sql """drop catalog if exists ${catalogName}""" +} + diff --git a/regression-test/suites/mtmv_p0/test_partition_refresh_mtmv.groovy b/regression-test/suites/mtmv_p0/test_partition_refresh_mtmv.groovy index 8e084091f4d15a..21296fc5878874 100644 --- a/regression-test/suites/mtmv_p0/test_partition_refresh_mtmv.groovy +++ b/regression-test/suites/mtmv_p0/test_partition_refresh_mtmv.groovy @@ -113,20 +113,17 @@ suite("test_partition_refresh_mtmv") { PROPERTIES ('replication_num' = '1') ; """ - try { - sql """ - CREATE MATERIALIZED VIEW ${mvName} - BUILD DEFERRED REFRESH AUTO ON MANUAL - partition by(`date`) - DISTRIBUTED BY RANDOM BUCKETS 2 - PROPERTIES ('replication_num' = '1') - AS - SELECT * FROM ${tableNameNum}; - """ - Assert.fail(); - } catch (Exception e) { - log.info(e.getMessage()) - } + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`date`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${tableNameNum}; + """ + sql """drop table if exists `${tableNameNum}`""" sql """drop materialized view if exists ${mvName};"""