From f683a18fb87cbeaeabed8cfa1c90ee509c658e42 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Thu, 21 Nov 2024 10:45:19 +0800 Subject: [PATCH 01/27] [feat](mtmv)mtmv support paimon partition refresh (#43959) PaimonUtil PaimonPartitionInfo PaimonSchemaCacheValue PaimonExternalTable use latest Previously, when using Paimon to create MTMV, it was not possible to perceive changes in partition lists and data, so only `refresh materialized view mv1 complete` could be used to force full refresh. This PR obtains the partition list of Paimon, the last update time of the partition, and the latest snapshotId of the table. Therefore, MTMV can be partitioned based on Paimon tables and perceive changes in data, automatically refreshing partitions mtmv support paimon partition refresh --- .../paimon/PaimonExternalTable.java | 132 +++++++++++++----- .../paimon/PaimonPartitionInfo.java | 11 +- .../paimon/PaimonSchemaCacheValue.java | 12 +- .../doris/datasource/paimon/PaimonUtil.java | 112 ++++----------- .../data/mtmv_p0/test_paimon_mtmv.out | 113 +++++++++++++++ .../suites/mtmv_p0/test_paimon_mtmv.groovy | 119 ++++++++++++++++ 6 files changed, 376 insertions(+), 123 deletions(-) create mode 100644 regression-test/data/mtmv_p0/test_paimon_mtmv.out create mode 100644 regression-test/suites/mtmv_p0/test_paimon_mtmv.groovy 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 ffe71bc30af3b7..635f3038bcf9a6 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,15 @@ package org.apache.doris.datasource.paimon; +import org.apache.doris.analysis.TableScanParams; +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; @@ -28,6 +34,14 @@ 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.mtmv.MTMVTimestampSnapshot; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ExternalAnalysisTask; @@ -36,36 +50,35 @@ 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; -import org.apache.paimon.CoreOptions; -import org.apache.paimon.catalog.Catalog; -import org.apache.paimon.data.InternalRow; -import org.apache.paimon.predicate.Predicate; -import org.apache.paimon.predicate.PredicateBuilder; +import org.apache.paimon.partition.Partition; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.DataTable; import org.apache.paimon.table.Table; import org.apache.paimon.table.source.Split; -import org.apache.paimon.table.system.SchemasTable; import org.apache.paimon.types.DataField; -import java.io.IOException; import java.util.Collections; 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 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() { @@ -75,16 +88,12 @@ 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()))); + return getOrFetchSnapshotCacheValue(snapshot).getSnapshot().getTable(); } public PaimonSchemaCacheValue getPaimonSchemaCacheValue(long schemaId) { @@ -141,6 +150,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)) { @@ -155,7 +187,47 @@ private boolean isPartitionInvalid(Optional snapshot) { } @Override - public MvccSnapshot loadSnapshot() { + public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, + Optional snapshot) + throws AnalysisException { + Partition paimonPartition = getOrFetchSnapshotCacheValue(snapshot).getPartitionInfo().getNameToPartition() + .get(partitionName); + if (paimonPartition == null) { + throw new AnalysisException("can not find partition: " + partitionName); + } + return new MTMVTimestampSnapshot(paimonPartition.lastFileCreationTime()); + } + + @Override + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) + throws AnalysisException { + return getTableSnapshot(snapshot); + } + + @Override + public MTMVSnapshotIf getTableSnapshot(Optional snapshot) throws AnalysisException { + PaimonSnapshotCacheValue paimonSnapshot = getOrFetchSnapshotCacheValue(snapshot); + return new MTMVSnapshotIdSnapshot(paimonSnapshot.getSnapshot().getSnapshotId()); + } + + @Override + public long getNewestUpdateVersionOrTime() { + return getPaimonSnapshotCacheValue().getPartitionInfo().getNameToPartition().values().stream() + .mapToLong(Partition::lastFileCreationTime).max().orElse(0); + } + + @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, Optional scanParams) { return new PaimonMvccSnapshot(getPaimonSnapshotCacheValue()); } @@ -179,10 +251,11 @@ public Optional initSchema(SchemaCacheKey key) { makeSureInitialized(); PaimonSchemaCacheKey paimonSchemaCacheKey = (PaimonSchemaCacheKey) key; try { - PaimonSchema schema = loadPaimonSchemaBySchemaId(paimonSchemaCacheKey); - List columns = schema.getFields(); + Table table = ((PaimonExternalCatalog) getCatalog()).getPaimonTable(key.getDbName(), name); + TableSchema tableSchema = ((DataTable) table).schemaManager().schema(paimonSchemaCacheKey.getSchemaId()); + List columns = tableSchema.fields(); List dorisColumns = Lists.newArrayListWithCapacity(columns.size()); - Set partitionColumnNames = Sets.newHashSet(schema.getPartitionKeys()); + Set partitionColumnNames = Sets.newHashSet(tableSchema.partitionKeys()); List partitionColumns = Lists.newArrayList(); for (DataField field : columns) { Column column = new Column(field.name().toLowerCase(), @@ -193,29 +266,13 @@ public Optional initSchema(SchemaCacheKey key) { partitionColumns.add(column); } } - return Optional.of(new PaimonSchemaCacheValue(dorisColumns, partitionColumns)); + return Optional.of(new PaimonSchemaCacheValue(dorisColumns, partitionColumns, tableSchema)); } catch (Exception e) { throw new CacheException("failed to initSchema for: %s.%s.%s.%s", null, getCatalog().getName(), key.getDbName(), key.getTblName(), paimonSchemaCacheKey.getSchemaId()); } - } - private PaimonSchema loadPaimonSchemaBySchemaId(PaimonSchemaCacheKey key) throws IOException { - Table table = ((PaimonExternalCatalog) getCatalog()).getPaimonTable(key.getDbName(), - name + Catalog.SYSTEM_TABLE_SPLITTER + SchemasTable.SCHEMAS); - PredicateBuilder builder = new PredicateBuilder(table.rowType()); - Predicate predicate = builder.equal(0, key.getSchemaId()); - // Adding predicates will also return excess data - List rows = PaimonUtil.read(table, new int[] {0, 1, 2}, predicate); - for (InternalRow row : rows) { - PaimonSchema schema = PaimonUtil.rowToSchema(row); - if (schema.getSchemaId() == key.getSchemaId()) { - return schema; - } - } - throw new CacheException("failed to initSchema for: %s.%s.%s.%s", - null, getCatalog().getName(), key.getDbName(), key.getTblName(), key.getSchemaId()); } private PaimonSchemaCacheValue getPaimonSchemaCacheValue(Optional snapshot) { @@ -231,4 +288,9 @@ private PaimonSnapshotCacheValue getOrFetchSnapshotCacheValue(Optional getSupportedSysTables() { + makeSureInitialized(); + return SupportedSysTables.PAIMON_SUPPORTED_SYS_TABLES; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonPartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonPartitionInfo.java index 88515a2510d2c7..a6339ef5155e15 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonPartitionInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonPartitionInfo.java @@ -20,20 +20,23 @@ import org.apache.doris.catalog.PartitionItem; import com.google.common.collect.Maps; +import org.apache.paimon.partition.Partition; import java.util.Map; public class PaimonPartitionInfo { + public static final PaimonPartitionInfo EMPTY = new PaimonPartitionInfo(); + private final Map nameToPartitionItem; - private final Map nameToPartition; + private final Map nameToPartition; - public PaimonPartitionInfo() { + private PaimonPartitionInfo() { this.nameToPartitionItem = Maps.newHashMap(); this.nameToPartition = Maps.newHashMap(); } public PaimonPartitionInfo(Map nameToPartitionItem, - Map nameToPartition) { + Map nameToPartition) { this.nameToPartitionItem = nameToPartitionItem; this.nameToPartition = nameToPartition; } @@ -42,7 +45,7 @@ public Map getNameToPartitionItem() { return nameToPartitionItem; } - public Map getNameToPartition() { + public Map getNameToPartition() { return nameToPartition; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheValue.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheValue.java index ccb530a3cbccc7..e931b52336ba8f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheValue.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheValue.java @@ -20,18 +20,28 @@ import org.apache.doris.catalog.Column; import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.paimon.schema.TableSchema; + import java.util.List; public class PaimonSchemaCacheValue extends SchemaCacheValue { private List partitionColumns; - public PaimonSchemaCacheValue(List schema, List partitionColumns) { + private TableSchema tableSchema; + // Caching TableSchema can reduce the reading of schema files and json parsing. + + public PaimonSchemaCacheValue(List schema, List partitionColumns, TableSchema tableSchema) { super(schema); this.partitionColumns = partitionColumns; + this.tableSchema = tableSchema; } public List getPartitionColumns() { return partitionColumns; } + + public TableSchema getTableSchema() { + return tableSchema; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtil.java index bbb1eaf5096520..caf4eecaa3a470 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtil.java @@ -36,17 +36,18 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.serializer.InternalRowSerializer; import org.apache.paimon.options.ConfigOption; +import org.apache.paimon.partition.Partition; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.reader.RecordReader; -import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.type.TypeReference; import org.apache.paimon.table.Table; import org.apache.paimon.table.source.ReadBuilder; import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.CharType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DecimalType; import org.apache.paimon.types.MapType; import org.apache.paimon.types.RowType; -import org.apache.paimon.utils.JsonSerdeUtil; +import org.apache.paimon.types.VarCharType; import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.Projection; @@ -69,7 +70,9 @@ public static List read( for (Pair, String> pair : dynamicOptions) { options.put(pair.getKey().key(), pair.getValue()); } - table = table.copy(options); + if (!options.isEmpty()) { + table = table.copy(options); + } ReadBuilder readBuilder = table.newReadBuilder(); if (projection != null) { readBuilder.withProjection(projection); @@ -89,71 +92,40 @@ public static List read( return rows; } + public static PaimonPartitionInfo generatePartitionInfo(List partitionColumns, + List paimonPartitions) { - /* - https://paimon.apache.org/docs/0.9/maintenance/system-tables/#partitions-table - +---------------+----------------+--------------------+--------------------+------------------------+ - | partition | record_count | file_size_in_bytes| file_count| last_update_time| - +---------------+----------------+--------------------+--------------------+------------------------+ - | [1] | 1 | 645 | 1 | 2024-06-24 10:25:57.400| - +---------------+----------------+--------------------+--------------------+------------------------+ - org.apache.paimon.table.system.PartitionsTable.TABLE_TYPE - public static final RowType TABLE_TYPE = - new RowType( - Arrays.asList( - new DataField(0, "partition", SerializationUtils.newStringType(true)), - new DataField(1, "record_count", new BigIntType(false)), - new DataField(2, "file_size_in_bytes", new BigIntType(false)), - new DataField(3, "file_count", new BigIntType(false)), - new DataField(4, "last_update_time", DataTypes.TIMESTAMP_MILLIS()))); - */ - public static PaimonPartition rowToPartition(InternalRow row) { - String partition = row.getString(0).toString(); - long recordCount = row.getLong(1); - long fileSizeInBytes = row.getLong(2); - long fileCount = row.getLong(3); - long lastUpdateTime = row.getTimestamp(4, 3).getMillisecond(); - return new PaimonPartition(partition, recordCount, fileSizeInBytes, fileCount, lastUpdateTime); - } + if (CollectionUtils.isEmpty(partitionColumns) || paimonPartitions.isEmpty()) { + return PaimonPartitionInfo.EMPTY; + } - public static PaimonPartitionInfo generatePartitionInfo(List partitionColumns, - List paimonPartitions) { Map nameToPartitionItem = Maps.newHashMap(); - Map nameToPartition = Maps.newHashMap(); + Map nameToPartition = Maps.newHashMap(); PaimonPartitionInfo partitionInfo = new PaimonPartitionInfo(nameToPartitionItem, nameToPartition); - if (CollectionUtils.isEmpty(partitionColumns)) { - return partitionInfo; - } - for (PaimonPartition paimonPartition : paimonPartitions) { - String partitionName = getPartitionName(partitionColumns, paimonPartition.getPartitionValues()); - nameToPartition.put(partitionName, paimonPartition); + + for (Partition partition : paimonPartitions) { + Map spec = partition.spec(); + StringBuilder sb = new StringBuilder(); + for (Map.Entry entry : spec.entrySet()) { + sb.append(entry.getKey()).append("=").append(entry.getValue()).append("/"); + } + if (sb.length() > 0) { + sb.deleteCharAt(sb.length() - 1); + } + String partitionName = sb.toString(); + nameToPartition.put(partitionName, partition); try { // partition values return by paimon api, may have problem, // to avoid affecting the query, we catch exceptions here nameToPartitionItem.put(partitionName, toListPartitionItem(partitionName, partitionColumns)); } catch (Exception e) { - LOG.warn("toListPartitionItem failed, partitionColumns: {}, partitionValues: {}", partitionColumns, - paimonPartition.getPartitionValues(), e); + LOG.warn("toListPartitionItem failed, partitionColumns: {}, partitionValues: {}", + partitionColumns, partition.spec(), e); } } return partitionInfo; } - private static String getPartitionName(List partitionColumns, String partitionValueStr) { - Preconditions.checkNotNull(partitionValueStr); - String[] partitionValues = partitionValueStr.replace("[", "").replace("]", "") - .split(","); - Preconditions.checkState(partitionColumns.size() == partitionValues.length); - StringBuilder sb = new StringBuilder(); - for (int i = 0; i < partitionColumns.size(); ++i) { - if (i != 0) { - sb.append("/"); - } - sb.append(partitionColumns.get(i).getName()).append("=").append(partitionValues[i]); - } - return sb.toString(); - } - public static ListPartitionItem toListPartitionItem(String partitionName, List partitionColumns) throws AnalysisException { List types = partitionColumns.stream() @@ -195,8 +167,10 @@ private static Type paimonPrimitiveTypeToDorisType(org.apache.paimon.types.DataT case TINYINT: return Type.TINYINT; case VARCHAR: - case BINARY: + return ScalarType.createVarcharType(((VarCharType) dataType).getLength()); case CHAR: + return ScalarType.createCharType(((CharType) dataType).getLength()); + case BINARY: case VARBINARY: return Type.STRING; case DECIMAL: @@ -251,32 +225,4 @@ private static Type paimonPrimitiveTypeToDorisType(org.apache.paimon.types.DataT public static Type paimonTypeToDorisType(org.apache.paimon.types.DataType type) { return paimonPrimitiveTypeToDorisType(type); } - - /** - * https://paimon.apache.org/docs/0.9/maintenance/system-tables/#schemas-table - * demo: - * 0 - * [{"id":0,"name":"user_id","type":"BIGINT NOT NULL"}, - * {"id":1,"name":"item_id","type":"BIGINT"}, - * {"id":2,"name":"behavior","type":"STRING"}, - * {"id":3,"name":"dt","type":"STRING NOT NULL"}, - * {"id":4,"name":"hh","type":"STRING NOT NULL"}] - * ["dt"] - * ["dt","hh","user_id"] - * {"owner":"hadoop","provider":"paimon"} - * 2024-12-03 15:38:14.734 - * - * @param row - * @return - */ - public static PaimonSchema rowToSchema(InternalRow row) { - long schemaId = row.getLong(0); - String fieldsStr = row.getString(1).toString(); - String partitionKeysStr = row.getString(2).toString(); - List fields = JsonSerdeUtil.fromJson(fieldsStr, new TypeReference>() { - }); - List partitionKeys = JsonSerdeUtil.fromJson(partitionKeysStr, new TypeReference>() { - }); - return new PaimonSchema(schemaId, fields, partitionKeys); - } } 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..c28b7cb7baca22 --- /dev/null +++ b/regression-test/data/mtmv_p0/test_paimon_mtmv.out @@ -0,0 +1,113 @@ +-- 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 + 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..f2989edbf6dfd6 --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_paimon_mtmv.groovy @@ -0,0 +1,119 @@ +// 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 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 + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + 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};""" + sql """drop catalog if exists ${catalogName}""" + +} + From 93f74b973a8fb9914dcd3636844199459a82e936 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Fri, 22 Nov 2024 14:53:14 +0800 Subject: [PATCH 02/27] [enhance](mtmv)Enable the MTMVRelatedTableIf interface to support mvcc (#44419) When using the mvcc table to obtain partition snapshots and other operations, the snapshotId parameter needs to be included --- .../src/main/java/org/apache/doris/catalog/MTMV.java | 9 +++++---- .../main/java/org/apache/doris/catalog/OlapTable.java | 1 + .../apache/doris/datasource/hive/HMSExternalTable.java | 1 + .../java/org/apache/doris/mtmv/MTMVPartitionUtil.java | 1 + .../java/org/apache/doris/mtmv/MTMVRelatedTableIf.java | 1 - .../org/apache/doris/mtmv/MTMVPartitionUtilTest.java | 1 + 6 files changed, 9 insertions(+), 5 deletions(-) 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..4e0dca91be620c 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 @@ -60,6 +60,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -361,8 +362,8 @@ public MTMVRefreshSnapshot getRefreshSnapshot() { * * @return mvPartitionName ==> mvPartitionKeyDesc */ - public Map generateMvPartitionDescs() { - Map mtmvItems = getAndCopyPartitionItems(); + public Map generateMvPartitionDescs() throws AnalysisException { + Map mtmvItems = getAndCopyPartitionItems(OptionalLong.empty()); Map result = Maps.newHashMap(); for (Entry entry : mtmvItems.entrySet()) { result.put(entry.getKey(), entry.getValue().toPartitionKeyDesc()); @@ -391,7 +392,7 @@ public Pair>, Map> calculateDoublyPartit Map baseToMv = Maps.newHashMap(); Map> relatedPartitionDescs = MTMVPartitionUtil .generateRelatedPartitionDescs(mvPartitionInfo, mvProperties); - Map mvPartitionItems = getAndCopyPartitionItems(); + Map mvPartitionItems = getAndCopyPartitionItems(OptionalLong.empty()); for (Entry entry : mvPartitionItems.entrySet()) { Set basePartitionNames = relatedPartitionDescs.getOrDefault(entry.getValue().toPartitionKeyDesc(), Sets.newHashSet()); @@ -424,7 +425,7 @@ public Map> calculatePartitionMappings() throws AnalysisExce Map> res = Maps.newHashMap(); Map> relatedPartitionDescs = MTMVPartitionUtil .generateRelatedPartitionDescs(mvPartitionInfo, mvProperties); - Map mvPartitionItems = getAndCopyPartitionItems(); + Map mvPartitionItems = getAndCopyPartitionItems(OptionalLong.empty()); for (Entry entry : mvPartitionItems.entrySet()) { res.put(entry.getKey(), relatedPartitionDescs.getOrDefault(entry.getValue().toPartitionKeyDesc(), Sets.newHashSet())); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index 06e29ae84c22a4..fa931652eef7cf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -109,6 +109,7 @@ import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; 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 35627273f93604..fe8d2389bdef24 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 @@ -92,6 +92,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; import java.util.stream.Collectors; 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..17fdbe6f867e77 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 @@ -53,6 +53,7 @@ import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; 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..974dd2b1b17063 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 @@ -98,7 +98,6 @@ MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext con */ MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) throws AnalysisException; - /** * Does the current type of table allow timed triggering * diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java index d90b43482eba1a..46787f96cfd77e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java @@ -40,6 +40,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; public class MTMVPartitionUtilTest { From 297c0ffda3ec63dec84c25ccbf1b0b7bfe826d25 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Thu, 19 Jun 2025 17:48:55 +0800 Subject: [PATCH 03/27] 1 --- .../test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java index 46787f96cfd77e..d90b43482eba1a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java @@ -40,7 +40,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.OptionalLong; import java.util.Set; public class MTMVPartitionUtilTest { From f43e7401613dc4164d59903cb3d185cca267ffc0 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Mon, 25 Nov 2024 19:35:12 +0800 Subject: [PATCH 04/27] [enhance](catalog)External partition prune return partitionName instead of partitionId (#44415) The partition ID of external data sources is meaningless, and some data sources only have partition names, so the return result of partition pruning is replaced with name instead of ID From ceb038a403f7a30cb947fc5d25713311b6640b9b Mon Sep 17 00:00:00 2001 From: zhangdong Date: Tue, 26 Nov 2024 11:51:28 +0800 Subject: [PATCH 05/27] [enhance](catalog)Unified external partition prune interface (#44567) Previously, external partition cropping only supported Hive. If you want to support other types of tables, you need to understand the internal processing logic of partition pruning. This PR abstracts the logic of partition pruning, and other tables can be implemented by simply covering a few methods of externalTable [opt](planner) Unified external partition prune interface --- .../src/main/java/org/apache/doris/datasource/ExternalTable.java | 1 + .../java/org/apache/doris/datasource/hive/HMSExternalTable.java | 1 - .../doris/nereids/rules/rewrite/PruneFileScanPartition.java | 1 + .../doris/nereids/trees/plans/logical/LogicalFileScan.java | 1 + 4 files changed, 3 insertions(+), 1 deletion(-) 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..ad20a19ec8cb42 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 @@ -58,6 +58,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; /** 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 fe8d2389bdef24..01f4f2b50e3403 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 @@ -46,7 +46,6 @@ 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; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java index e99906f5e13dc4..bb3c17268325a0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java @@ -36,6 +36,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.OptionalLong; import java.util.function.Function; import java.util.stream.Collectors; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java index 1f5f71f7bafe59..e6f75ef80bea24 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java @@ -37,6 +37,7 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.OptionalLong; /** * Logical file scan for external catalog. From 3123d161632623b84a9894e240a3afe510be2724 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Thu, 28 Nov 2024 10:22:20 +0800 Subject: [PATCH 06/27] [feat](mtmv)Unified external table interface supporting partition refresh and partition pruning (#44673) - Add `MvccTable` to represent a table that supports querying specified version data - Add the `MvccSnapshot` interface to store snapshot information of mvcc at a certain moment in time - Add the `MvccSnapshot` parameter to the method of the `MTMVRelatedTableIf `interface to retrieve data of a specified version - Partition pruning related methods combined with the `MvccSnapshot` parameter are used to obtain partition information for a specified version - Load the snapshot information of mvccTable at the beginning of the query plan and store it in StatementContext Unified external table interface supporting partition refresh and partition pruning --- .../src/main/java/org/apache/doris/catalog/MTMV.java | 7 +++---- .../src/main/java/org/apache/doris/catalog/OlapTable.java | 1 - .../java/org/apache/doris/datasource/ExternalTable.java | 1 - .../org/apache/doris/datasource/hive/HMSExternalTable.java | 1 - .../main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java | 1 - .../java/org/apache/doris/mtmv/MTMVRelatedTableIf.java | 1 + .../java/org/apache/doris/nereids/CascadesContext.java | 1 + .../java/org/apache/doris/nereids/StatementContext.java | 6 ++++-- .../nereids/rules/rewrite/PruneFileScanPartition.java | 2 +- .../doris/nereids/trees/plans/logical/LogicalFileScan.java | 1 - 10 files changed, 10 insertions(+), 12 deletions(-) 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 4e0dca91be620c..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 @@ -60,7 +60,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Optional; -import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -363,7 +362,7 @@ public MTMVRefreshSnapshot getRefreshSnapshot() { * @return mvPartitionName ==> mvPartitionKeyDesc */ public Map generateMvPartitionDescs() throws AnalysisException { - Map mtmvItems = getAndCopyPartitionItems(OptionalLong.empty()); + Map mtmvItems = getAndCopyPartitionItems(); Map result = Maps.newHashMap(); for (Entry entry : mtmvItems.entrySet()) { result.put(entry.getKey(), entry.getValue().toPartitionKeyDesc()); @@ -392,7 +391,7 @@ public Pair>, Map> calculateDoublyPartit Map baseToMv = Maps.newHashMap(); Map> relatedPartitionDescs = MTMVPartitionUtil .generateRelatedPartitionDescs(mvPartitionInfo, mvProperties); - Map mvPartitionItems = getAndCopyPartitionItems(OptionalLong.empty()); + Map mvPartitionItems = getAndCopyPartitionItems(); for (Entry entry : mvPartitionItems.entrySet()) { Set basePartitionNames = relatedPartitionDescs.getOrDefault(entry.getValue().toPartitionKeyDesc(), Sets.newHashSet()); @@ -425,7 +424,7 @@ public Map> calculatePartitionMappings() throws AnalysisExce Map> res = Maps.newHashMap(); Map> relatedPartitionDescs = MTMVPartitionUtil .generateRelatedPartitionDescs(mvPartitionInfo, mvProperties); - Map mvPartitionItems = getAndCopyPartitionItems(OptionalLong.empty()); + Map mvPartitionItems = getAndCopyPartitionItems(); for (Entry entry : mvPartitionItems.entrySet()) { res.put(entry.getKey(), relatedPartitionDescs.getOrDefault(entry.getValue().toPartitionKeyDesc(), Sets.newHashSet())); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index fa931652eef7cf..06e29ae84c22a4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -109,7 +109,6 @@ import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; -import java.util.OptionalLong; import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; 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 ad20a19ec8cb42..30bf48c3d8b76a 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 @@ -58,7 +58,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.OptionalLong; import java.util.Set; /** 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 01f4f2b50e3403..fbca9840da12d0 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 @@ -91,7 +91,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Optional; -import java.util.OptionalLong; import java.util.Set; import java.util.stream.Collectors; 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 17fdbe6f867e77..8da00cdeeb3421 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 @@ -53,7 +53,6 @@ import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; -import java.util.OptionalLong; import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; 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 974dd2b1b17063..c4261aa78f10be 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 @@ -98,6 +98,7 @@ MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext con */ MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) throws AnalysisException; + /** * Does the current type of table allow timed triggering * diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index 1486f03e269b13..866734c6cf3c7d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -56,6 +56,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; +import org.apache.commons.collections.MapUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; 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 c9e1e23d0b1212..670d235b9a8539 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,8 @@ package org.apache.doris.nereids; import org.apache.doris.analysis.StatementBase; +import org.apache.doris.analysis.TableScanParams; +import org.apache.doris.analysis.TableSnapshot; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.View; @@ -646,13 +648,13 @@ public void addPlannerHook(PlannerHook plannerHook) { /** * Load snapshot information of mvcc */ - public void loadSnapshots() { + public void loadSnapshots(Optional tableSnapshot, Optional scanParams) { 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, scanParams)); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java index bb3c17268325a0..3d92037073ab0c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java @@ -36,7 +36,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.OptionalLong; +import java.util.Optional; import java.util.function.Function; import java.util.stream.Collectors; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java index e6f75ef80bea24..1f5f71f7bafe59 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java @@ -37,7 +37,6 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; -import java.util.OptionalLong; /** * Logical file scan for external catalog. From df36768e7050eae1690da626605c44efe530f6e7 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Thu, 5 Dec 2024 14:15:59 +0800 Subject: [PATCH 07/27] [feat](mtmv)external table support partition rewrite (#44998) Previously, transparent rewriting of the external table could only be done as a whole or without rewriting. Now supports partial partition rewriting and direct lookup of the base table for some partitions. mtmv partition rewrite support external table --- .../mv/AbstractMaterializedViewRule.java | 7 +- .../rules/exploration/mv/StructInfo.java | 18 ++-- .../commands/UpdateMvByPartitionCommand.java | 8 +- .../data/mtmv_p0/test_hive_rewrite_mtmv.out | 31 +++++++ .../mtmv_p0/test_hive_rewrite_mtmv.groovy | 89 +++++++++++++++++++ 5 files changed, 137 insertions(+), 16 deletions(-) create mode 100644 regression-test/data/mtmv_p0/test_hive_rewrite_mtmv.out create mode 100644 regression-test/suites/mtmv_p0/test_hive_rewrite_mtmv.groovy 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 80200af0a51e00..cd29c5572e2b1d 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 @@ -456,17 +456,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/StructInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java index 2e2119efe7176c..365360e06b096c 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/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/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}""" + } +} + From 2f5dfdc18aa911952861a31ca13676b18ab2de20 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 11 Dec 2024 17:48:28 +0800 Subject: [PATCH 08/27] [feat](mtmv)use real snapshot instead of optional.empty() (#45273) In the previous PR, a snapshot of the table was obtained and stored in the statementContext at the beginning of the query. The modification of this PR is to ensure that the same metadata is used during the query process. When calling the relevant interface, snapshot needs to be obtained from statementContext as a parameter and passed to the relevant method Related PR: #44911 #44673 --- .../doris/mtmv/MTMVPartitionExprDateTrunc.java | 3 ++- .../org/apache/doris/mtmv/MTMVPartitionInfo.java | 6 ++++-- .../org/apache/doris/mtmv/MTMVPartitionUtil.java | 13 ++++++------- .../mtmv/MTMVRelatedPartitionDescInitGenerator.java | 5 +++-- .../MTMVRelatedPartitionDescRollUpGenerator.java | 4 ++-- .../rules/exploration/mv/MaterializedViewUtils.java | 6 ++++-- 6 files changed, 21 insertions(+), 16 deletions(-) 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..480d14c804dfd1 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; @@ -52,7 +53,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Objects; -import java.util.Optional; import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -327,7 +327,7 @@ public static boolean isSyncWithPartitions(MTMVRefreshContext context, String mt } 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 +546,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 +567,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/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/nereids/rules/exploration/mv/MaterializedViewUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java index b504298bba6e3b..ddae11fb73518b 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; @@ -449,13 +450,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.getColumn().get(); Expr definExpr = mvReferenceColumn.getDefineExpr(); if (definExpr instanceof SlotRef) { From 756d67bfc92c79b024f2d037115021d97e39124a Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 11 Dec 2024 10:53:08 +0800 Subject: [PATCH 09/27] [feat](mtmv)Paimon queries the data in the cache instead of querying the latest data (#44911) Problem Summary: - add `PaimonMetadataCacheMgr` in `ExternalMetaCacheMgr` to manage snapshotCache of paimon table - move paimonSchemaCache to PaimonMetadataCacheMgr, and add schemaId as part of key - PaimonExternalTable overrides the methods in ExternalTable and supports partition pruning - PaimonExternalTable implements the MvcTable interface, supporting the retrieval of snapshot data from the cache during queries to avoid cache refreshes that may result in different versions of metadata being used in a single query - MTMVTask retrieves snapshot data of mvccTable before the task starts to avoid cache refresh that may result in different versions of metadata being used in a single refresh task Paimon queries the data in the cache instead of querying the latest data behavior changes of query paimon table: - FE has just started and is query the latest data - Paimon data has changed, Doris is still query the previous data - After the snapshot cache expires, Doris will query the latest data - desc paimon; The schema corresponding to the snapshotId in the snapshot cache is displayed --- .../paimon/PaimonMetadataCache.java | 48 ++++------ .../rules/rewrite/PruneFileScanPartition.java | 1 - .../data/mtmv_p0/test_paimon_rewrite_mtmv.out | 16 ++++ .../mtmv_p0/test_paimon_rewrite_mtmv.groovy | 95 +++++++++++++++++++ 4 files changed, 129 insertions(+), 31 deletions(-) create mode 100644 regression-test/data/mtmv_p0/test_paimon_rewrite_mtmv.out create mode 100644 regression-test/suites/mtmv_p0/test_paimon_rewrite_mtmv.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCache.java index 109394fabded6a..dfc1e6a1c25c61 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCache.java @@ -26,19 +26,19 @@ import org.apache.doris.datasource.ExternalMetaCacheMgr; import com.github.benmanes.caffeine.cache.LoadingCache; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.commons.collections.CollectionUtils; -import org.apache.paimon.catalog.Catalog; -import org.apache.paimon.data.InternalRow; +import org.apache.paimon.CoreOptions; +import org.apache.paimon.Snapshot; +import org.apache.paimon.partition.Partition; import org.apache.paimon.table.Table; -import org.apache.paimon.table.system.PartitionsTable; -import org.apache.paimon.table.system.SnapshotsTable; import org.jetbrains.annotations.NotNull; import java.io.IOException; +import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.OptionalLong; import java.util.concurrent.ExecutorService; @@ -75,39 +75,27 @@ private PaimonSnapshotCacheValue loadSnapshot(PaimonSnapshotCacheKey key) { private PaimonPartitionInfo loadPartitionInfo(PaimonSnapshotCacheKey key, List partitionColumns) throws IOException, AnalysisException { if (CollectionUtils.isEmpty(partitionColumns)) { - return new PaimonPartitionInfo(); + return PaimonPartitionInfo.EMPTY; } - List paimonPartitions = loadPartitions(key); + List paimonPartitions = ((PaimonExternalCatalog) key.getCatalog()) + .getPaimonPartitions(key.getDbName(), key.getTableName()); return PaimonUtil.generatePartitionInfo(partitionColumns, paimonPartitions); } - private List loadPartitions(PaimonSnapshotCacheKey key) - throws IOException { - Table table = ((PaimonExternalCatalog) key.getCatalog()).getPaimonTable(key.getDbName(), - key.getTableName() + Catalog.SYSTEM_TABLE_SPLITTER + PartitionsTable.PARTITIONS); - List rows = PaimonUtil.read(table, null, null); - List res = Lists.newArrayListWithCapacity(rows.size()); - for (InternalRow row : rows) { - res.add(PaimonUtil.rowToPartition(row)); - } - return res; - } - private PaimonSnapshot loadLatestSnapshot(PaimonSnapshotCacheKey key) throws IOException { - Table table = ((PaimonExternalCatalog) key.getCatalog()).getPaimonTable(key.getDbName(), - key.getTableName() + Catalog.SYSTEM_TABLE_SPLITTER + SnapshotsTable.SNAPSHOTS); + Table table = ((PaimonExternalCatalog) key.getCatalog()).getPaimonTable(key.getDbName(), key.getTableName()); + Table snapshotTable = table; // snapshotId and schemaId - List rows = PaimonUtil.read(table, new int[] {0, 1}, null); - long latestSnapshotId = 0L; + Long latestSnapshotId = PaimonSnapshot.INVALID_SNAPSHOT_ID; long latestSchemaId = 0L; - for (InternalRow row : rows) { - long snapshotId = row.getLong(0); - if (snapshotId > latestSnapshotId) { - latestSnapshotId = snapshotId; - latestSchemaId = row.getLong(1); - } + Optional optionalSnapshot = table.latestSnapshot(); + if (optionalSnapshot.isPresent()) { + latestSnapshotId = optionalSnapshot.get().id(); + latestSchemaId = table.snapshot(latestSnapshotId).schemaId(); + snapshotTable = + table.copy(Collections.singletonMap(CoreOptions.SCAN_SNAPSHOT_ID.key(), latestSnapshotId.toString())); } - return new PaimonSnapshot(latestSnapshotId, latestSchemaId); + return new PaimonSnapshot(latestSnapshotId, latestSchemaId, snapshotTable); } public void invalidateCatalogCache(long catalogId) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java index 3d92037073ab0c..e99906f5e13dc4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java @@ -36,7 +36,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.function.Function; import java.util.stream.Collectors; 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/mtmv_p0/test_paimon_rewrite_mtmv.groovy b/regression-test/suites/mtmv_p0/test_paimon_rewrite_mtmv.groovy new file mode 100644 index 00000000000000..985443875c7b26 --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_paimon_rewrite_mtmv.groovy @@ -0,0 +1,95 @@ +// 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}") + + //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}""" +} + From e82fc3b4c33fce6223617a367b43cfea76564980 Mon Sep 17 00:00:00 2001 From: James Date: Mon, 16 Dec 2024 15:34:39 +0800 Subject: [PATCH 10/27] [feature](mtmv)Support iceberg partition refresh. (#44726) Previously, when using Iceberg to create MTMV, it was not possible to perceive changes in partition lists and data, so only ```refresh materialized view mv1 complete ```could be used to force full refresh. This PR obtains the partition list of Iceberg, the last update time of the partition, and the latest snapshotId of the table. Therefore, MTMV can be partition based on Iceberg tables and perceive changes in data, automatically refreshing partitions For now, we only support single partition column table and the partition transform must one of hour, day, month or year. Will support Identity transform soon. Issue Number: close #xxx Related PR: #xxx Problem Summary: None --- .../doris/catalog/RangePartitionItem.java | 7 +- .../iceberg/IcebergExternalTable.java | 439 +++++++++++++++++- .../datasource/iceberg/IcebergPartition.java | 82 ++++ .../iceberg/IcebergPartitionInfo.java | 71 +++ .../iceberg/IcebergSchemaCacheValue.java | 50 ++ .../doris/job/extensions/mtmv/MTMVTask.java | 8 + .../apache/doris/mtmv/MTMVPartitionUtil.java | 2 +- .../apache/doris/mtmv/MTMVRelatedTableIf.java | 9 + .../iceberg/IcebergExternalTableTest.java | 238 ++++++++++ .../iceberg/IcebergPartitionInfoTest.java | 53 +++ .../data/mtmv_p0/test_iceberg_mtmv.out | 98 ++++ .../suites/mtmv_p0/test_iceberg_mtmv.groovy | 147 ++++++ 12 files changed, 1200 insertions(+), 4 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergPartition.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergPartitionInfo.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSchemaCacheValue.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergExternalTableTest.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergPartitionInfoTest.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionItem.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionItem.java index cad6ca38130420..96bf0097c28a51 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionItem.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionItem.java @@ -65,9 +65,14 @@ public boolean isDefaultPartition() { @Override public PartitionKeyDesc toPartitionKeyDesc() { - return PartitionKeyDesc.createFixed( + if (partitionKeyRange.hasLowerBound()) { + return PartitionKeyDesc.createFixed( PartitionInfo.toPartitionValue(partitionKeyRange.lowerEndpoint()), PartitionInfo.toPartitionValue(partitionKeyRange.upperEndpoint())); + } else { + // For null partition value. + return PartitionKeyDesc.createLessThan(PartitionInfo.toPartitionValue(partitionKeyRange.upperEndpoint())); + } } @Override 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 a357532f2bf1f8..32103153238619 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,9 +17,24 @@ package org.apache.doris.datasource.iceberg; +import org.apache.doris.analysis.PartitionValue; 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.ExternalTable; import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.doris.datasource.mvcc.MvccSnapshot; +import org.apache.doris.mtmv.MTMVBaseTableIf; +import org.apache.doris.mtmv.MTMVRefreshContext; +import org.apache.doris.mtmv.MTMVRelatedTableIf; +import org.apache.doris.mtmv.MTMVSnapshotIf; +import org.apache.doris.mtmv.MTMVVersionSnapshot; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ExternalAnalysisTask; @@ -28,13 +43,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 { + + 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) { @@ -52,9 +107,50 @@ 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))); + table = IcebergUtils.getIcebergTable(catalog, dbName, name); + List schema = IcebergUtils.getSchema(catalog, dbName, name); + Snapshot snapshot = table.currentSnapshot(); + if (snapshot == null) { + LOG.debug("Table {} is empty", name); + return Optional.of(new IcebergSchemaCacheValue(schema, null, -1, null)); + } + long snapshotId = snapshot.snapshotId(); + partitionColumns = null; + IcebergPartitionInfo partitionInfo = null; + if (isValidRelatedTable()) { + PartitionSpec spec = table.spec(); + partitionColumns = Lists.newArrayList(); + + // For iceberg table, we only support table with 1 partition column as RelatedTable. + // So we use spec.fields().get(0) to get the partition column. + Types.NestedField col = table.schema().findField(spec.fields().get(0).sourceId()); + for (Column c : schema) { + if (c.getName().equalsIgnoreCase(col.name())) { + partitionColumns.add(c); + break; + } + } + Preconditions.checkState(partitionColumns.size() == 1, + "Support 1 partition column for iceberg table, but found " + partitionColumns.size()); + try { + partitionInfo = loadPartitionInfo(); + } catch (AnalysisException e) { + LOG.warn("Failed to load iceberg table {} partition info.", name, e); + } + } + return Optional.of(new IcebergSchemaCacheValue(schema, partitionColumns, snapshotId, partitionInfo)); } @Override @@ -91,4 +187,343 @@ public long fetchRowCount() { public Table getIcebergTable() { return IcebergUtils.getIcebergTable(getCatalog(), getDbName(), getName()); } + + @Override + public void beforeMTMVRefresh(MTMV mtmv) throws DdlException { + Env.getCurrentEnv().getRefreshManager() + .refreshTable(getCatalog().getName(), getDbName(), getName(), true); + } + + @Override + public Map getAndCopyPartitionItems(Optional snapshot) { + return Maps.newHashMap(getPartitionInfoFromCache().getNameToPartitionItem()); + } + + private IcebergPartitionInfo getPartitionInfoFromCache() { + makeSureInitialized(); + Optional schemaCacheValue = getSchemaCacheValue(); + if (!schemaCacheValue.isPresent()) { + return new IcebergPartitionInfo(); + } + return ((IcebergSchemaCacheValue) schemaCacheValue.get()).getPartitionInfo(); + } + + @Override + public PartitionType getPartitionType(Optional snapshot) { + makeSureInitialized(); + return isValidRelatedTable() ? PartitionType.RANGE : PartitionType.UNPARTITIONED; + } + + @Override + public Set getPartitionColumnNames(Optional snapshot) throws DdlException { + return getPartitionColumnsFromCache().stream().map(Column::getName).collect(Collectors.toSet()); + } + + @Override + public List getPartitionColumns(Optional snapshot) { + return getPartitionColumnsFromCache(); + } + + private List getPartitionColumnsFromCache() { + makeSureInitialized(); + Optional schemaCacheValue = getSchemaCacheValue(); + return schemaCacheValue + .map(cacheValue -> ((IcebergSchemaCacheValue) cacheValue).getPartitionColumns()) + .orElseGet(Lists::newArrayList); + } + + @Override + public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, + Optional snapshot) throws AnalysisException { + long latestSnapshotId = getPartitionInfoFromCache().getLatestSnapshotId(partitionName); + if (latestSnapshotId <= 0) { + throw new AnalysisException("can not find partition: " + partitionName); + } + return new MTMVVersionSnapshot(latestSnapshotId); + } + + @Override + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) + throws AnalysisException { + return new MTMVVersionSnapshot(getLatestSnapshotIdFromCache()); + } + + public long getLatestSnapshotIdFromCache() throws AnalysisException { + makeSureInitialized(); + Optional schemaCacheValue = getSchemaCacheValue(); + if (!schemaCacheValue.isPresent()) { + throw new AnalysisException("Can't find schema cache of table " + name); + } + return ((IcebergSchemaCacheValue) schemaCacheValue.get()).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() { + if (isValidRelatedTableCached) { + return isValidRelatedTable; + } + isValidRelatedTable = false; + Set allFields = Sets.newHashSet(); + 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; + } + + protected IcebergPartitionInfo loadPartitionInfo() throws AnalysisException { + List icebergPartitions = loadIcebergPartition(); + Map nameToPartition = Maps.newHashMap(); + Map nameToPartitionItem = Maps.newHashMap(); + 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); + PartitionItem item = new RangePartitionItem(partitionRange); + nameToPartitionItem.put(partition.getPartitionName(), item); + } + Map> partitionNameMap = mergeOverlapPartitions(nameToPartitionItem); + return new IcebergPartitionInfo(nameToPartitionItem, nameToPartition, partitionNameMap); + } + + public List loadIcebergPartition() { + PartitionsTable partitionsTable = (PartitionsTable) MetadataTableUtils + .createMetadataTableInstance(table, MetadataTableType.PARTITIONS); + List partitions = Lists.newArrayList(); + try (CloseableIterable tasks = partitionsTable.newScan().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 + 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) + throws AnalysisException { + // For NULL value, create a lessThan partition for it. + if (value == null) { + PartitionKey nullKey = PartitionKey.createPartitionKey( + Lists.newArrayList(new PartitionValue("0000-01-02")), partitionColumns); + return Range.lessThan(nullKey); + } + 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; + } } 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/IcebergSchemaCacheValue.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSchemaCacheValue.java new file mode 100644 index 00000000000000..e1fde8049fe1ad --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSchemaCacheValue.java @@ -0,0 +1,50 @@ +// 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; + private final IcebergPartitionInfo partitionInfo; + private final long snapshotId; + + public IcebergSchemaCacheValue(List schema, List partitionColumns, + long snapshotId, IcebergPartitionInfo partitionInfo) { + super(schema); + this.partitionColumns = partitionColumns; + this.snapshotId = snapshotId; + this.partitionInfo = partitionInfo; + } + + public List getPartitionColumns() { + return partitionColumns; + } + + public IcebergPartitionInfo getPartitionInfo() { + return partitionInfo; + } + + public long getSnapshotId() { + return snapshotId; + } +} 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 198ba21891832b..c74091c11d4ba2 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; @@ -203,6 +204,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); 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 480d14c804dfd1..47f61357c21f02 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 @@ -320,7 +320,7 @@ 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; 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/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..80d0a7c2429df3 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergExternalTableTest.java @@ -0,0 +1,238 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.iceberg; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.catalog.PartitionKey; +import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.catalog.RangePartitionItem; +import org.apache.doris.common.AnalysisException; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Range; +import mockit.Expectations; +import mockit.Mocked; +import mockit.Verifications; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.transforms.Days; +import org.apache.iceberg.transforms.Hours; +import org.apache.iceberg.transforms.Months; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class IcebergExternalTableTest { + + @Test + public void testIsSupportedPartitionTable(@Mocked org.apache.iceberg.Table icebergTable, + @Mocked PartitionSpec spec, + @Mocked PartitionField field, + @Mocked Schema schema) { + IcebergExternalTable table = new IcebergExternalTable(1, "1", "2", null); + Map specs = Maps.newHashMap(); + // Test null + specs.put(0, null); + new Expectations() {{ + icebergTable.specs(); + result = specs; + }}; + table.setTable(icebergTable); + Assertions.assertFalse(table.isValidRelatedTableCached()); + Assertions.assertFalse(table.isValidRelatedTable()); + new Verifications() {{ + icebergTable.specs(); + times = 1; + }}; + Assertions.assertTrue(table.isValidRelatedTableCached()); + Assertions.assertFalse(table.validRelatedTableCache()); + + // Test spec fields are empty. + specs.put(0, spec); + table.setIsValidRelatedTableCached(false); + Assertions.assertFalse(table.isValidRelatedTableCached()); + new Expectations() {{ + icebergTable.specs(); + result = specs; + }}; + List fields = Lists.newArrayList(); + new Expectations() {{ + spec.fields(); + result = fields; + }}; + Assertions.assertFalse(table.isValidRelatedTable()); + new Verifications() {{ + spec.fields(); + times = 1; + }}; + Assertions.assertTrue(table.isValidRelatedTableCached()); + Assertions.assertFalse(table.validRelatedTableCache()); + + // Test true + fields.add(field); + table.setIsValidRelatedTableCached(false); + Assertions.assertFalse(table.isValidRelatedTableCached()); + new Expectations() { + { + icebergTable.schema(); + result = schema; + + schema.findColumnName(anyInt); + result = "col1"; + } + }; + new Expectations() {{ + field.transform(); + result = new Hours(); + }}; + Assertions.assertTrue(table.isValidRelatedTable()); + Assertions.assertTrue(table.isValidRelatedTableCached()); + Assertions.assertTrue(table.validRelatedTableCache()); + new Verifications() {{ + schema.findColumnName(anyInt); + times = 1; + }}; + new Expectations() {{ + field.transform(); + result = new Days(); + }}; + table.setIsValidRelatedTableCached(false); + Assertions.assertFalse(table.isValidRelatedTableCached()); + Assertions.assertTrue(table.isValidRelatedTable()); + new Expectations() {{ + field.transform(); + result = new Months(); + }}; + table.setIsValidRelatedTableCached(false); + Assertions.assertFalse(table.isValidRelatedTableCached()); + Assertions.assertTrue(table.isValidRelatedTable()); + Assertions.assertTrue(table.isValidRelatedTableCached()); + Assertions.assertTrue(table.validRelatedTableCache()); + } + + @Test + public void testGetPartitionRange() throws AnalysisException { + IcebergExternalTable table = new IcebergExternalTable(1, "1", "2", null); + Column c = new Column("ts", PrimitiveType.DATETIMEV2); + List partitionColumns = Lists.newArrayList(c); + table.setPartitionColumns(partitionColumns); + + // Test null partition value + Range nullRange = table.getPartitionRange(null, "hour"); + Assertions.assertFalse(nullRange.hasLowerBound()); + Assertions.assertEquals("0000-01-02 00:00:00", + nullRange.upperEndpoint().getPartitionValuesAsStringList().get(0)); + + // Test hour transform. + Range hour = table.getPartitionRange("100", "hour"); + PartitionKey lowKey = hour.lowerEndpoint(); + PartitionKey upKey = hour.upperEndpoint(); + Assertions.assertEquals("1970-01-05 04:00:00", lowKey.getPartitionValuesAsStringList().get(0)); + Assertions.assertEquals("1970-01-05 05:00:00", upKey.getPartitionValuesAsStringList().get(0)); + + // Test day transform. + Range day = table.getPartitionRange("100", "day"); + lowKey = day.lowerEndpoint(); + upKey = day.upperEndpoint(); + Assertions.assertEquals("1970-04-11 00:00:00", lowKey.getPartitionValuesAsStringList().get(0)); + Assertions.assertEquals("1970-04-12 00:00:00", upKey.getPartitionValuesAsStringList().get(0)); + + // Test month transform. + Range month = table.getPartitionRange("100", "month"); + lowKey = month.lowerEndpoint(); + upKey = month.upperEndpoint(); + Assertions.assertEquals("1978-05-01 00:00:00", lowKey.getPartitionValuesAsStringList().get(0)); + Assertions.assertEquals("1978-06-01 00:00:00", upKey.getPartitionValuesAsStringList().get(0)); + + // Test year transform. + Range year = table.getPartitionRange("100", "year"); + lowKey = year.lowerEndpoint(); + upKey = year.upperEndpoint(); + Assertions.assertEquals("2070-01-01 00:00:00", lowKey.getPartitionValuesAsStringList().get(0)); + Assertions.assertEquals("2071-01-01 00:00:00", upKey.getPartitionValuesAsStringList().get(0)); + + // Test unsupported transform + Exception exception = Assertions.assertThrows(RuntimeException.class, () -> { + table.getPartitionRange("100", "bucket"); + }); + Assertions.assertEquals("Unsupported transform bucket", exception.getMessage()); + } + + @Test + public void testSortRange() throws AnalysisException { + IcebergExternalTable table = new IcebergExternalTable(1, "1", "2", null); + Column c = new Column("c", PrimitiveType.DATETIMEV2); + table.setPartitionColumns(Lists.newArrayList(c)); + PartitionItem nullRange = new RangePartitionItem(table.getPartitionRange(null, "hour")); + PartitionItem year1970 = new RangePartitionItem(table.getPartitionRange("0", "year")); + PartitionItem year1971 = new RangePartitionItem(table.getPartitionRange("1", "year")); + PartitionItem month197002 = new RangePartitionItem(table.getPartitionRange("1", "month")); + PartitionItem month197103 = new RangePartitionItem(table.getPartitionRange("14", "month")); + PartitionItem month197204 = new RangePartitionItem(table.getPartitionRange("27", "month")); + PartitionItem day19700202 = new RangePartitionItem(table.getPartitionRange("32", "day")); + PartitionItem day19730101 = new RangePartitionItem(table.getPartitionRange("1096", "day")); + Map map = Maps.newHashMap(); + map.put("nullRange", nullRange); + map.put("year1970", year1970); + map.put("year1971", year1971); + map.put("month197002", month197002); + map.put("month197103", month197103); + map.put("month197204", month197204); + map.put("day19700202", day19700202); + map.put("day19730101", day19730101); + List> entries = table.sortPartitionMap(map); + Assertions.assertEquals(8, entries.size()); + Assertions.assertEquals("nullRange", entries.get(0).getKey()); + Assertions.assertEquals("year1970", entries.get(1).getKey()); + Assertions.assertEquals("month197002", entries.get(2).getKey()); + Assertions.assertEquals("day19700202", entries.get(3).getKey()); + Assertions.assertEquals("year1971", entries.get(4).getKey()); + Assertions.assertEquals("month197103", entries.get(5).getKey()); + Assertions.assertEquals("month197204", entries.get(6).getKey()); + Assertions.assertEquals("day19730101", entries.get(7).getKey()); + + Map> stringSetMap = table.mergeOverlapPartitions(map); + Assertions.assertEquals(2, stringSetMap.size()); + Assertions.assertTrue(stringSetMap.containsKey("year1970")); + Assertions.assertTrue(stringSetMap.containsKey("year1971")); + + Set names1970 = stringSetMap.get("year1970"); + Assertions.assertEquals(3, names1970.size()); + Assertions.assertTrue(names1970.contains("year1970")); + Assertions.assertTrue(names1970.contains("month197002")); + Assertions.assertTrue(names1970.contains("day19700202")); + + Set names1971 = stringSetMap.get("year1971"); + Assertions.assertEquals(2, names1971.size()); + Assertions.assertTrue(names1971.contains("year1971")); + Assertions.assertTrue(names1971.contains("month197103")); + + Assertions.assertEquals(5, map.size()); + Assertions.assertTrue(map.containsKey("nullRange")); + Assertions.assertTrue(map.containsKey("year1970")); + Assertions.assertTrue(map.containsKey("year1971")); + Assertions.assertTrue(map.containsKey("month197204")); + Assertions.assertTrue(map.containsKey("day19730101")); + } +} 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/regression-test/data/mtmv_p0/test_iceberg_mtmv.out b/regression-test/data/mtmv_p0/test_iceberg_mtmv.out index d0b47e27fffae5..c9d9799da81300 100644 --- a/regression-test/data/mtmv_p0/test_iceberg_mtmv.out +++ b/regression-test/data/mtmv_p0/test_iceberg_mtmv.out @@ -5,3 +5,101 @@ -- !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 + +-- !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 + diff --git a/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy b/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy index 3004f1677bded6..59cf1173acb46b 100644 --- a/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy +++ b/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy @@ -65,5 +65,152 @@ 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 hivePrefix = "hive2"; + String hms_port = context.config.otherConfigs.get(hivePrefix + "HmsPort") + String hdfs_port = context.config.otherConfigs.get(hivePrefix + "HdfsPort") + String default_fs = "hdfs://${externalEnvIp}:${hdfs_port}" + String warehouse = "${default_fs}/warehouse" + + 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" + sql """drop catalog if exists ${catalog_name} """ + sql """create catalog if not exists ${catalog_name} properties ( + 'type'='iceberg', + 'iceberg.catalog.type'='hms', + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', + 'fs.defaultFS' = '${default_fs}', + 'warehouse' = '${warehouse}', + 'use_meta_cache' = 'true' + );""" + + 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) + 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) + 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) + qt_test_ts_refresh3 """select * from ${mvName1} order by value""" + + sql """REFRESH MATERIALIZED VIEW ${mvName1} auto""" + waitingMTMVTaskFinishedByMvName(mvName1) + 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) + qt_test_ts_refresh5 """select * from ${mvName1} order by value""" + + sql """REFRESH MATERIALIZED VIEW ${mvName1} auto""" + waitingMTMVTaskFinishedByMvName(mvName1) + 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) + qt_test_ts_refresh_null """select * from ${mvName1} order by value""" + + 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) + 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) + 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) + qt_test_d_refresh3 "select * from ${mvName2} order by value" + sql """REFRESH MATERIALIZED VIEW ${mvName2} partitions(p_20240901_20241001);""" + waitingMTMVTaskFinishedByMvName(mvName2) + 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) + qt_test_d_refresh5 "select * from ${mvName2} order by value" + + 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}""" + + sql """ drop catalog if exists ${catalog_name} """ + } } From 9c27a90a085db86c3157c967efe9b62abfb6ded3 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Mon, 23 Dec 2024 19:09:17 +0800 Subject: [PATCH 11/27] [enhance](mtmv)MTMV allow paimon table has multi partition keys (#45652) - MTMV allow paimon table has multi partition keys - add case --- .../paimon/run01.sql | 20 +-- .../info/MTMVPartitionDefinition.java | 6 +- .../data/mtmv_p0/test_paimon_mtmv.out | 29 ++++ .../mtmv_p0/test_paimon_olap_rewrite_mtmv.out | 79 +++++++++ .../suites/mtmv_p0/test_paimon_mtmv.groovy | 155 +++++++++++++++++- .../test_paimon_olap_rewrite_mtmv.groovy | 115 +++++++++++++ .../mtmv_p0/test_paimon_rewrite_mtmv.groovy | 3 + .../test_partition_refresh_mtmv.groovy | 25 ++- 8 files changed, 403 insertions(+), 29 deletions(-) create mode 100644 regression-test/data/mtmv_p0/test_paimon_olap_rewrite_mtmv.out create mode 100644 regression-test/suites/mtmv_p0/test_paimon_olap_rewrite_mtmv.groovy 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/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/regression-test/data/mtmv_p0/test_paimon_mtmv.out b/regression-test/data/mtmv_p0/test_paimon_mtmv.out index c28b7cb7baca22..ba6fc06c1d2491 100644 --- a/regression-test/data/mtmv_p0/test_paimon_mtmv.out +++ b/regression-test/data/mtmv_p0/test_paimon_mtmv.out @@ -111,3 +111,32 @@ false -- !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 + 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/suites/mtmv_p0/test_paimon_mtmv.groovy b/regression-test/suites/mtmv_p0/test_paimon_mtmv.groovy index f2989edbf6dfd6..48d63e03ec3db5 100644 --- a/regression-test/suites/mtmv_p0/test_paimon_mtmv.groovy +++ b/regression-test/suites/mtmv_p0/test_paimon_mtmv.groovy @@ -25,6 +25,24 @@ suite("test_paimon_mtmv", "p0,external,mtmv,external_docker,external_docker_dori 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") @@ -99,8 +117,10 @@ suite("test_paimon_mtmv", "p0,external,mtmv,external_docker,external_docker_dori sql """ CREATE MATERIALIZED VIEW ${mvName} BUILD DEFERRED REFRESH AUTO ON MANUAL - DISTRIBUTED BY RANDOM BUCKETS 2 - PROPERTIES ('replication_num' = '1') + 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; """ @@ -113,6 +133,137 @@ suite("test_paimon_mtmv", "p0,external,mtmv,external_docker,external_docker_dori 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};""" + 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 index 985443875c7b26..22a94d46635169 100644 --- a/regression-test/suites/mtmv_p0/test_paimon_rewrite_mtmv.groovy +++ b/regression-test/suites/mtmv_p0/test_paimon_rewrite_mtmv.groovy @@ -75,6 +75,9 @@ suite("test_paimon_rewrite_mtmv", "p0,external,mtmv,external_docker,external_doc 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 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};""" From db1dd02fede8ebeb5c436c5ce79f2425fb7a77f8 Mon Sep 17 00:00:00 2001 From: James Date: Tue, 24 Dec 2024 11:58:51 +0800 Subject: [PATCH 12/27] [feature](mtmv)Support iceberg mtmv query. (#45659) 1. Implement MvccTable interface for IcebertExternalTable 2. IcebergExternalTable overrides the methods in ExternalTable and supports partition pruning 3. Add snapshot cache in IcebergMetadataCache to store IcebergExternalTable partition infos. Issue Number: close #xxx Related PR: #xxx Problem Summary: None --- .../doris/catalog/RangePartitionItem.java | 7 +- .../datasource/hive/HMSExternalTable.java | 7 +- .../iceberg/IcebergExternalTable.java | 171 +++--- .../iceberg/IcebergMetadataCache.java | 42 +- .../iceberg/IcebergMvccSnapshot.java | 32 ++ .../iceberg/IcebergSchemaCacheKey.java | 55 ++ .../iceberg/IcebergSchemaCacheValue.java | 15 +- .../datasource/iceberg/IcebergSnapshot.java | 36 ++ .../iceberg/IcebergSnapshotCacheValue.java | 37 ++ .../datasource/iceberg/IcebergUtils.java | 522 +++++++++++++++++- .../iceberg/IcebergExternalTableTest.java | 48 +- .../data/mtmv_p0/test_iceberg_mtmv.out | 15 + .../suites/mtmv_p0/test_iceberg_mtmv.groovy | 56 ++ 13 files changed, 933 insertions(+), 110 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMvccSnapshot.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSchemaCacheKey.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSnapshot.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergSnapshotCacheValue.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionItem.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionItem.java index 96bf0097c28a51..cad6ca38130420 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionItem.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionItem.java @@ -65,14 +65,9 @@ public boolean isDefaultPartition() { @Override public PartitionKeyDesc toPartitionKeyDesc() { - if (partitionKeyRange.hasLowerBound()) { - return PartitionKeyDesc.createFixed( + return PartitionKeyDesc.createFixed( PartitionInfo.toPartitionValue(partitionKeyRange.lowerEndpoint()), PartitionInfo.toPartitionValue(partitionKeyRange.upperEndpoint())); - } else { - // For null partition value. - return PartitionKeyDesc.createLessThan(PartitionInfo.toPartitionValue(partitionKeyRange.upperEndpoint())); - } } @Override 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 fbca9840da12d0..5967bd15b02cf7 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 @@ -37,6 +37,7 @@ import org.apache.doris.datasource.TablePartitionValues; 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.MvccSnapshot; import org.apache.doris.mtmv.MTMVBaseTableIf; @@ -549,10 +550,8 @@ public Optional initSchema() { } } - 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() { 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 32103153238619..713ec94fd1bdd1 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 @@ -27,9 +27,14 @@ 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.mtmv.MTMVBaseTableIf; import org.apache.doris.mtmv.MTMVRefreshContext; import org.apache.doris.mtmv.MTMVRelatedTableIf; @@ -77,7 +82,7 @@ import java.util.Set; import java.util.stream.Collectors; -public class IcebergExternalTable extends ExternalTable implements MTMVRelatedTableIf, MTMVBaseTableIf { +public class IcebergExternalTable extends ExternalTable implements MTMVRelatedTableIf, MTMVBaseTableIf, MvccTable { public static final String YEAR = "year"; public static final String MONTH = "month"; @@ -118,39 +123,23 @@ public void setPartitionColumns(List partitionColumns) { } @Override - public Optional initSchema() { - table = IcebergUtils.getIcebergTable(catalog, dbName, name); - List schema = IcebergUtils.getSchema(catalog, dbName, name); - Snapshot snapshot = table.currentSnapshot(); - if (snapshot == null) { - LOG.debug("Table {} is empty", name); - return Optional.of(new IcebergSchemaCacheValue(schema, null, -1, null)); - } - long snapshotId = snapshot.snapshotId(); - partitionColumns = null; - IcebergPartitionInfo partitionInfo = null; - if (isValidRelatedTable()) { - PartitionSpec spec = table.spec(); - partitionColumns = Lists.newArrayList(); - - // For iceberg table, we only support table with 1 partition column as RelatedTable. - // So we use spec.fields().get(0) to get the partition column. - Types.NestedField col = table.schema().findField(spec.fields().get(0).sourceId()); + 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())) { - partitionColumns.add(c); + tmpColumns.add(c); break; } } - Preconditions.checkState(partitionColumns.size() == 1, - "Support 1 partition column for iceberg table, but found " + partitionColumns.size()); - try { - partitionInfo = loadPartitionInfo(); - } catch (AnalysisException e) { - LOG.warn("Failed to load iceberg table {} partition info.", name, e); - } } - return Optional.of(new IcebergSchemaCacheValue(schema, partitionColumns, snapshotId, partitionInfo)); + partitionColumns = tmpColumns; + return Optional.of(new IcebergSchemaCacheValue(schema, partitionColumns)); } @Override @@ -188,6 +177,11 @@ 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 { Env.getCurrentEnv().getRefreshManager() @@ -196,46 +190,36 @@ public void beforeMTMVRefresh(MTMV mtmv) throws DdlException { @Override public Map getAndCopyPartitionItems(Optional snapshot) { - return Maps.newHashMap(getPartitionInfoFromCache().getNameToPartitionItem()); + return Maps.newHashMap(getOrFetchSnapshotCacheValue(snapshot).getPartitionInfo().getNameToPartitionItem()); } - private IcebergPartitionInfo getPartitionInfoFromCache() { - makeSureInitialized(); - Optional schemaCacheValue = getSchemaCacheValue(); - if (!schemaCacheValue.isPresent()) { - return new IcebergPartitionInfo(); - } - return ((IcebergSchemaCacheValue) schemaCacheValue.get()).getPartitionInfo(); + @Override + public Map getNameToPartitionItems(Optional snapshot) { + return getOrFetchSnapshotCacheValue(snapshot).getPartitionInfo().getNameToPartitionItem(); } @Override public PartitionType getPartitionType(Optional snapshot) { - makeSureInitialized(); return isValidRelatedTable() ? PartitionType.RANGE : PartitionType.UNPARTITIONED; } @Override public Set getPartitionColumnNames(Optional snapshot) throws DdlException { - return getPartitionColumnsFromCache().stream().map(Column::getName).collect(Collectors.toSet()); + return getPartitionColumns(snapshot).stream().map(Column::getName).collect(Collectors.toSet()); } @Override public List getPartitionColumns(Optional snapshot) { - return getPartitionColumnsFromCache(); - } - - private List getPartitionColumnsFromCache() { - makeSureInitialized(); - Optional schemaCacheValue = getSchemaCacheValue(); - return schemaCacheValue - .map(cacheValue -> ((IcebergSchemaCacheValue) cacheValue).getPartitionColumns()) - .orElseGet(Lists::newArrayList); + 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 { - long latestSnapshotId = getPartitionInfoFromCache().getLatestSnapshotId(partitionName); + IcebergSnapshotCacheValue snapshotValue = getOrFetchSnapshotCacheValue(snapshot); + long latestSnapshotId = snapshotValue.getPartitionInfo().getLatestSnapshotId(partitionName); if (latestSnapshotId <= 0) { throw new AnalysisException("can not find partition: " + partitionName); } @@ -245,16 +229,9 @@ public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshCont @Override public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) throws AnalysisException { - return new MTMVVersionSnapshot(getLatestSnapshotIdFromCache()); - } - - public long getLatestSnapshotIdFromCache() throws AnalysisException { makeSureInitialized(); - Optional schemaCacheValue = getSchemaCacheValue(); - if (!schemaCacheValue.isPresent()) { - throw new AnalysisException("Can't find schema cache of table " + name); - } - return ((IcebergSchemaCacheValue) schemaCacheValue.get()).getSnapshotId(); + IcebergSnapshotCacheValue snapshotValue = getOrFetchSnapshotCacheValue(snapshot); + return new MTMVVersionSnapshot(snapshotValue.getSnapshot().getSnapshotId()); } @Override @@ -269,11 +246,13 @@ public boolean isPartitionColumnAllowNull() { */ @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; @@ -300,14 +279,62 @@ public boolean isValidRelatedTable() { return isValidRelatedTable; } - protected IcebergPartitionInfo loadPartitionInfo() throws AnalysisException { - List icebergPartitions = loadIcebergPartition(); + @Override + public MvccSnapshot loadSnapshot() { + 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); + Range partitionRange = getPartitionRange( + partition.getPartitionValues().get(0), transform, partitionColumns); PartitionItem item = new RangePartitionItem(partitionRange); nameToPartitionItem.put(partition.getPartitionName(), item); } @@ -315,11 +342,11 @@ protected IcebergPartitionInfo loadPartitionInfo() throws AnalysisException { return new IcebergPartitionInfo(nameToPartitionItem, nameToPartition, partitionNameMap); } - public List loadIcebergPartition() { + public List loadIcebergPartition(long snapshotId) { PartitionsTable partitionsTable = (PartitionsTable) MetadataTableUtils .createMetadataTableInstance(table, MetadataTableType.PARTITIONS); List partitions = Lists.newArrayList(); - try (CloseableIterable tasks = partitionsTable.newScan().planFiles()) { + try (CloseableIterable tasks = partitionsTable.newScan().useSnapshot(snapshotId).planFiles()) { for (FileScanTask task : tasks) { CloseableIterable rows = task.asDataTask().rows(); for (StructLike row : rows) { @@ -345,6 +372,7 @@ public IcebergPartition generateIcebergPartition(StructLike row) { // 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); @@ -383,13 +411,14 @@ public IcebergPartition generateIcebergPartition(StructLike row) { } @VisibleForTesting - public Range getPartitionRange(String value, String transform) + public Range getPartitionRange(String value, String transform, List partitionColumns) throws AnalysisException { - // For NULL value, create a lessThan partition for it. + // For NULL value, create a minimum partition for it. if (value == null) { - PartitionKey nullKey = PartitionKey.createPartitionKey( - Lists.newArrayList(new PartitionValue("0000-01-02")), partitionColumns); - return Range.lessThan(nullKey); + 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; @@ -526,4 +555,12 @@ public boolean validRelatedTableCache() { public void setIsValidRelatedTableCached(boolean isCached) { this.isValidRelatedTableCached = isCached; } + + private IcebergSnapshotCacheValue getOrFetchSnapshotCacheValue(Optional snapshot) { + if (snapshot.isPresent()) { + return ((IcebergMvccSnapshot) snapshot.get()).getSnapshotCacheValue(); + } else { + return getIcebergSnapshotCacheValue(); + } + } } 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/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 index e1fde8049fe1ad..ccfcaab0c7261d 100644 --- 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 @@ -25,26 +25,13 @@ public class IcebergSchemaCacheValue extends SchemaCacheValue { private final List partitionColumns; - private final IcebergPartitionInfo partitionInfo; - private final long snapshotId; - public IcebergSchemaCacheValue(List schema, List partitionColumns, - long snapshotId, IcebergPartitionInfo partitionInfo) { + public IcebergSchemaCacheValue(List schema, List partitionColumns) { super(schema); this.partitionColumns = partitionColumns; - this.snapshotId = snapshotId; - this.partitionInfo = partitionInfo; } public List getPartitionColumns() { return partitionColumns; } - - public IcebergPartitionInfo getPartitionInfo() { - return partitionInfo; - } - - public long getSnapshotId() { - return snapshotId; - } } 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 76e9781b78b693..716d12f6a13d7b 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 @@ -31,52 +31,96 @@ import org.apache.doris.analysis.LiteralExpr; import org.apache.doris.analysis.NullLiteral; import org.apache.doris.analysis.PartitionDesc; +import org.apache.doris.analysis.PartitionValue; import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.StringLiteral; import org.apache.doris.analysis.Subquery; +import org.apache.doris.analysis.TableScanParams; +import org.apache.doris.analysis.TableSnapshot; import org.apache.doris.catalog.ArrayType; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.MapType; +import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.catalog.PartitionKey; +import org.apache.doris.catalog.RangePartitionItem; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.StructField; import org.apache.doris.catalog.StructType; import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.Type; +import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; import org.apache.doris.common.info.SimpleTableInfo; import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.datasource.CacheException; import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.ExternalSchemaCache; +import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.doris.datasource.iceberg.source.IcebergTableQueryInfo; +import org.apache.doris.datasource.mvcc.MvccSnapshot; +import org.apache.doris.datasource.mvcc.MvccUtil; 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.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.commons.lang3.exception.ExceptionUtils; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.ManifestFile; +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.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.expressions.And; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.ManifestEvaluator; import org.apache.iceberg.expressions.Not; import org.apache.iceberg.expressions.Or; +import org.apache.iceberg.expressions.Projections; import org.apache.iceberg.expressions.Unbound; import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.types.Type.TypeID; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.LocationUtil; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.StructProjection; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.io.IOException; +import java.time.DateTimeException; +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.List; import java.util.Locale; import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.regex.Pattern; import java.util.stream.Collectors; /** @@ -107,6 +151,18 @@ 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; // means an empty table + public static final long NEWEST_SCHEMA_ID = -1; + + 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 static final Pattern SNAPSHOT_ID = Pattern.compile("\\d+"); + public static Expression convertToIcebergExpr(Expr expr, Schema schema) { if (expr == null) { return null; @@ -573,11 +629,18 @@ private static org.apache.iceberg.Table getIcebergTableInternal(ExternalCatalog /** * 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 == NEWEST_SCHEMA_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) { @@ -683,10 +746,465 @@ public static HiveCatalog createIcebergHiveCatalog(ExternalCatalog externalCatal hiveCatalog.setConf(externalCatalog.getConfiguration()); Map catalogProperties = externalCatalog.getProperties(); + if (!catalogProperties.containsKey(HiveCatalog.LIST_ALL_TABLES)) { + // This configuration will display all tables (including non-Iceberg type tables), + // which can save the time of obtaining table objects. + // Later, type checks will be performed when loading the table. + catalogProperties.put(HiveCatalog.LIST_ALL_TABLES, "true"); + } String metastoreUris = catalogProperties.getOrDefault(HMSProperties.HIVE_METASTORE_URIS, ""); catalogProperties.put(CatalogProperties.URI, metastoreUris); hiveCatalog.initialize(name, catalogProperties); return hiveCatalog; } + + // Retrieve the manifest files that match the query based on partitions in filter + public static CloseableIterable getMatchingManifest( + List dataManifests, + Map specsById, + Expression dataFilter) { + LoadingCache evalCache = Caffeine.newBuilder() + .build( + specId -> { + PartitionSpec spec = specsById.get(specId); + return ManifestEvaluator.forPartitionFilter( + Expressions.and( + Expressions.alwaysTrue(), + Projections.inclusive(spec, true).project(dataFilter)), + spec, + true); + }); + + CloseableIterable matchingManifests = CloseableIterable.filter( + CloseableIterable.withNoopClose(dataManifests), + manifest -> evalCache.get(manifest.partitionSpecId()).eval(manifest)); + + matchingManifests = + CloseableIterable.filter( + matchingManifests, + manifest -> manifest.hasAddedFiles() || manifest.hasExistingFiles()); + + return matchingManifests; + } + + // get snapshot id from query like 'for version/time as of' or '@branch/@tag' + public static IcebergTableQueryInfo getQuerySpecSnapshot( + Table table, + Optional queryTableSnapshot, + Optional scanParams) throws UserException { + + Preconditions.checkArgument( + queryTableSnapshot.isPresent() || isIcebergBranchOrTag(scanParams), + "should spec version or time or branch or tag"); + + // not support `select * from tb@branch/tag(b) for version/time as of ...` + Preconditions.checkArgument( + !(queryTableSnapshot.isPresent() && isIcebergBranchOrTag(scanParams)), + "could not spec a version/time with tag/branch"); + + // solve @branch/@tag + if (scanParams.isPresent()) { + String refName; + TableScanParams params = scanParams.get(); + if (!params.getMapParams().isEmpty()) { + refName = params.getMapParams().get("name"); + } else { + refName = params.getListParams().get(0); + } + SnapshotRef snapshotRef = table.refs().get(refName); + if (params.isBranch()) { + if (snapshotRef == null || !snapshotRef.isBranch()) { + throw new UserException("Table " + table.name() + " does not have branch named " + refName); + } + } else { + if (snapshotRef == null || !snapshotRef.isTag()) { + throw new UserException("Table " + table.name() + " does not have tag named " + refName); + } + } + return new IcebergTableQueryInfo( + snapshotRef.snapshotId(), + refName, + SnapshotUtil.schemaFor(table, refName).schemaId()); + } + + // solve version/time as of + String value = queryTableSnapshot.get().getValue(); + TableSnapshot.VersionType type = queryTableSnapshot.get().getType(); + if (type == TableSnapshot.VersionType.VERSION) { + if (SNAPSHOT_ID.matcher(value).matches()) { + long snapshotId = Long.parseLong(value); + Snapshot snapshot = table.snapshot(snapshotId); + if (snapshot == null) { + throw new UserException("Table " + table.name() + " does not have snapshotId " + value); + } + return new IcebergTableQueryInfo( + snapshotId, + null, + snapshot.schemaId() + ); + } + + if (!table.refs().containsKey(value)) { + throw new UserException("Table " + table.name() + " does not have tag or branch named " + value); + } + return new IcebergTableQueryInfo( + table.refs().get(value).snapshotId(), + value, + SnapshotUtil.schemaFor(table, value).schemaId() + ); + } else { + long timestamp = TimeUtils.timeStringToLong(value, TimeUtils.getTimeZone()); + if (timestamp < 0) { + throw new DateTimeException("can't parse time: " + value); + } + long snapshotId = SnapshotUtil.snapshotIdAsOfTime(table, timestamp); + return new IcebergTableQueryInfo( + snapshotId, + null, + table.snapshot(snapshotId).schemaId() + ); + } + } + + public static boolean isIcebergBranchOrTag(Optional scanParams) { + if (scanParams == null || !scanParams.isPresent()) { + return false; + } + TableScanParams params = scanParams.get(); + if (params.isBranch() || params.isTag()) { + if (!params.getMapParams().isEmpty()) { + Preconditions.checkArgument( + params.getMapParams().containsKey("name"), + "must contain key 'name' in params" + ); + } else { + Preconditions.checkArgument( + params.getListParams().size() == 1 + && params.getListParams().get(0) != null, + "must contain a branch/tag name in params" + ); + } + return true; + } + return false; + } + + // read schema from external schema cache + public static IcebergSchemaCacheValue getSchemaCacheValue( + ExternalCatalog catalog, String dbName, String name, 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 static IcebergSnapshot getLastedIcebergSnapshot(ExternalCatalog catalog, String dbName, String tbName) { + Table table = IcebergUtils.getIcebergTable(catalog, dbName, tbName); + Snapshot snapshot = table.currentSnapshot(); + long snapshotId = snapshot == null ? IcebergUtils.UNKNOWN_SNAPSHOT_ID : snapshot.snapshotId(); + return new IcebergSnapshot(snapshotId, table.schema().schemaId()); + } + + public static IcebergPartitionInfo loadPartitionInfo( + ExternalCatalog catalog, String dbName, String tbName, long snapshotId) throws AnalysisException { + // snapshotId == UNKNOWN_SNAPSHOT_ID means this is an empty table, haven't contained any snapshot yet. + if (snapshotId == IcebergUtils.UNKNOWN_SNAPSHOT_ID) { + return IcebergPartitionInfo.empty(); + } + Table table = getIcebergTable(catalog, dbName, tbName); + List icebergPartitions = loadIcebergPartition(table, snapshotId); + Map nameToPartition = Maps.newHashMap(); + Map nameToPartitionItem = Maps.newHashMap(); + + List partitionColumns = IcebergUtils.getSchemaCacheValue( + catalog, dbName, tbName, 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); + } + + private static List loadIcebergPartition(Table table, 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(table, row)); + } + } + } catch (IOException e) { + LOG.warn("Failed to get Iceberg table {} partition info.", table.name(), e); + } + return partitions; + } + + private static IcebergPartition generateIcebergPartition(Table table, 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 + 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 static 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. + */ + @VisibleForTesting + public static 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) + */ + @VisibleForTesting + public static 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; + } + } + + public static IcebergSnapshotCacheValue getIcebergSnapshotCacheValue( + Optional tableSnapshot, + ExternalCatalog catalog, + String dbName, + String tbName, + Optional scanParams) { + IcebergSnapshotCacheValue snapshotCache = Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache() + .getSnapshotCache(catalog, dbName, tbName); + if (tableSnapshot.isPresent() || IcebergUtils.isIcebergBranchOrTag(scanParams)) { + // If a snapshot is specified, + // use the specified snapshot and the corresponding schema(not the latest schema). + Table icebergTable = getIcebergTable(catalog, dbName, tbName); + IcebergTableQueryInfo info; + try { + info = getQuerySpecSnapshot(icebergTable, tableSnapshot, scanParams); + } catch (UserException e) { + throw new RuntimeException(e); + } + return new IcebergSnapshotCacheValue( + IcebergPartitionInfo.empty(), + new IcebergSnapshot(info.getSnapshotId(), info.getSchemaId())); + } else { + // Otherwise, use the latest snapshot and the latest schema. + return snapshotCache; + } + } + + // 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)); + } + + public static List getIcebergSchema( + TableIf tableIf, + ExternalCatalog catalog, + String dbName, + String tbName) { + Optional snapshotFromContext = MvccUtil.getSnapshotFromContext(tableIf); + IcebergSnapshotCacheValue cacheValue = + IcebergUtils.getOrFetchSnapshotCacheValue(snapshotFromContext, catalog, dbName, tbName); + return IcebergUtils.getSchemaCacheValue( + catalog, dbName, tbName, cacheValue.getSnapshot().getSchemaId()) + .getSchema(); + } + + public static IcebergSnapshotCacheValue getOrFetchSnapshotCacheValue( + Optional snapshot, + ExternalCatalog catalog, + String dbName, + String tbName) { + if (snapshot.isPresent()) { + return ((IcebergMvccSnapshot) snapshot.get()).getSnapshotCacheValue(); + } else { + return IcebergUtils.getIcebergSnapshotCacheValue( + Optional.empty(), catalog, dbName, tbName, Optional.empty()); + } + } } 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 index 80d0a7c2429df3..3ba4804e52279c 100644 --- 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 @@ -28,17 +28,21 @@ import com.google.common.collect.Maps; import com.google.common.collect.Range; import mockit.Expectations; +import mockit.Mock; +import mockit.MockUp; import mockit.Mocked; import mockit.Verifications; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.transforms.Days; import org.apache.iceberg.transforms.Hours; import org.apache.iceberg.transforms.Months; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Set; @@ -52,6 +56,16 @@ public void testIsSupportedPartitionTable(@Mocked org.apache.iceberg.Table icebe @Mocked Schema schema) { IcebergExternalTable table = new IcebergExternalTable(1, "1", "2", null); Map specs = Maps.newHashMap(); + new MockUp() { + @Mock + private void makeSureInitialized() { + } + + @Mock + public Table getIcebergTable() { + return icebergTable; + } + }; // Test null specs.put(0, null); new Expectations() {{ @@ -139,34 +153,35 @@ public void testGetPartitionRange() throws AnalysisException { table.setPartitionColumns(partitionColumns); // Test null partition value - Range nullRange = table.getPartitionRange(null, "hour"); - Assertions.assertFalse(nullRange.hasLowerBound()); - Assertions.assertEquals("0000-01-02 00:00:00", + Range nullRange = table.getPartitionRange(null, "hour", partitionColumns); + Assertions.assertEquals("0000-01-01 00:00:00", + nullRange.lowerEndpoint().getPartitionValuesAsStringList().get(0)); + Assertions.assertEquals("0000-01-01 00:00:01", nullRange.upperEndpoint().getPartitionValuesAsStringList().get(0)); // Test hour transform. - Range hour = table.getPartitionRange("100", "hour"); + Range hour = table.getPartitionRange("100", "hour", partitionColumns); PartitionKey lowKey = hour.lowerEndpoint(); PartitionKey upKey = hour.upperEndpoint(); Assertions.assertEquals("1970-01-05 04:00:00", lowKey.getPartitionValuesAsStringList().get(0)); Assertions.assertEquals("1970-01-05 05:00:00", upKey.getPartitionValuesAsStringList().get(0)); // Test day transform. - Range day = table.getPartitionRange("100", "day"); + Range day = table.getPartitionRange("100", "day", partitionColumns); lowKey = day.lowerEndpoint(); upKey = day.upperEndpoint(); Assertions.assertEquals("1970-04-11 00:00:00", lowKey.getPartitionValuesAsStringList().get(0)); Assertions.assertEquals("1970-04-12 00:00:00", upKey.getPartitionValuesAsStringList().get(0)); // Test month transform. - Range month = table.getPartitionRange("100", "month"); + Range month = table.getPartitionRange("100", "month", partitionColumns); lowKey = month.lowerEndpoint(); upKey = month.upperEndpoint(); Assertions.assertEquals("1978-05-01 00:00:00", lowKey.getPartitionValuesAsStringList().get(0)); Assertions.assertEquals("1978-06-01 00:00:00", upKey.getPartitionValuesAsStringList().get(0)); // Test year transform. - Range year = table.getPartitionRange("100", "year"); + Range year = table.getPartitionRange("100", "year", partitionColumns); lowKey = year.lowerEndpoint(); upKey = year.upperEndpoint(); Assertions.assertEquals("2070-01-01 00:00:00", lowKey.getPartitionValuesAsStringList().get(0)); @@ -174,7 +189,7 @@ public void testGetPartitionRange() throws AnalysisException { // Test unsupported transform Exception exception = Assertions.assertThrows(RuntimeException.class, () -> { - table.getPartitionRange("100", "bucket"); + table.getPartitionRange("100", "bucket", partitionColumns); }); Assertions.assertEquals("Unsupported transform bucket", exception.getMessage()); } @@ -183,15 +198,16 @@ public void testGetPartitionRange() throws AnalysisException { public void testSortRange() throws AnalysisException { IcebergExternalTable table = new IcebergExternalTable(1, "1", "2", null); Column c = new Column("c", PrimitiveType.DATETIMEV2); + ArrayList columns = Lists.newArrayList(c); table.setPartitionColumns(Lists.newArrayList(c)); - PartitionItem nullRange = new RangePartitionItem(table.getPartitionRange(null, "hour")); - PartitionItem year1970 = new RangePartitionItem(table.getPartitionRange("0", "year")); - PartitionItem year1971 = new RangePartitionItem(table.getPartitionRange("1", "year")); - PartitionItem month197002 = new RangePartitionItem(table.getPartitionRange("1", "month")); - PartitionItem month197103 = new RangePartitionItem(table.getPartitionRange("14", "month")); - PartitionItem month197204 = new RangePartitionItem(table.getPartitionRange("27", "month")); - PartitionItem day19700202 = new RangePartitionItem(table.getPartitionRange("32", "day")); - PartitionItem day19730101 = new RangePartitionItem(table.getPartitionRange("1096", "day")); + PartitionItem nullRange = new RangePartitionItem(table.getPartitionRange(null, "hour", columns)); + PartitionItem year1970 = new RangePartitionItem(table.getPartitionRange("0", "year", columns)); + PartitionItem year1971 = new RangePartitionItem(table.getPartitionRange("1", "year", columns)); + PartitionItem month197002 = new RangePartitionItem(table.getPartitionRange("1", "month", columns)); + PartitionItem month197103 = new RangePartitionItem(table.getPartitionRange("14", "month", columns)); + PartitionItem month197204 = new RangePartitionItem(table.getPartitionRange("27", "month", columns)); + PartitionItem day19700202 = new RangePartitionItem(table.getPartitionRange("32", "day", columns)); + PartitionItem day19730101 = new RangePartitionItem(table.getPartitionRange("1096", "day", columns)); Map map = Maps.newHashMap(); map.put("nullRange", nullRange); map.put("year1970", year1970); diff --git a/regression-test/data/mtmv_p0/test_iceberg_mtmv.out b/regression-test/data/mtmv_p0/test_iceberg_mtmv.out index c9d9799da81300..483ac0957e6f67 100644 --- a/regression-test/data/mtmv_p0/test_iceberg_mtmv.out +++ b/regression-test/data/mtmv_p0/test_iceberg_mtmv.out @@ -103,3 +103,18 @@ 2024-09-30 6 2024-10-28 7 +-- !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 + diff --git a/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy b/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy index 59cf1173acb46b..aee80d8d1693a4 100644 --- a/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy +++ b/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy @@ -83,6 +83,7 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_ 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 if not exists ${catalog_name} properties ( 'type'='iceberg', @@ -210,6 +211,61 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_ 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) + 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) + 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 """ drop catalog if exists ${catalog_name} """ } } From 48072276422d1905b97fdc3a6b7d63dd5d17bfe1 Mon Sep 17 00:00:00 2001 From: James Date: Mon, 6 Jan 2025 15:54:27 +0800 Subject: [PATCH 13/27] [regression](mtmv)Support show iceberg external table partition. Add more test case for iceberg mtmv. (#46257) ### What problem does this PR solve? Support show iceberg external table partition. We convert iceberg partition to doris range partition in IcebergExternalTable. This PR add show partition function for IcebergExternalTable, this make it possible to add regression test. Issue Number: close #xxx Related PR: #xxx Problem Summary: ### Release note None --- .../iceberg/run08.sql | 106 ++++++++++++++++++ .../doris/analysis/ShowPartitionsStmt.java | 18 ++- .../iceberg/IcebergExternalTable.java | 2 +- .../org/apache/doris/qe/ShowExecutor.java | 40 +++++++ .../data/mtmv_p0/test_iceberg_mtmv.out | 19 ++++ .../suites/mtmv_p0/test_iceberg_mtmv.groovy | 67 +++++++++-- 6 files changed, 237 insertions(+), 15 deletions(-) create mode 100644 docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run08.sql 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/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/datasource/iceberg/IcebergExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java index 713ec94fd1bdd1..2feab480d7edb8 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 @@ -556,7 +556,7 @@ public void setIsValidRelatedTableCached(boolean isCached) { this.isValidRelatedTableCached = isCached; } - private IcebergSnapshotCacheValue getOrFetchSnapshotCacheValue(Optional snapshot) { + public IcebergSnapshotCacheValue getOrFetchSnapshotCacheValue(Optional snapshot) { if (snapshot.isPresent()) { return ((IcebergMvccSnapshot) snapshot.get()).getSnapshotCacheValue(); } else { 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 2087fd283c3078..a864796814a6f8 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 @@ -140,6 +140,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; @@ -204,6 +206,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; @@ -257,6 +260,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; @@ -1901,6 +1905,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); } @@ -1984,6 +1990,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/regression-test/data/mtmv_p0/test_iceberg_mtmv.out b/regression-test/data/mtmv_p0/test_iceberg_mtmv.out index 483ac0957e6f67..433cc85d3325eb 100644 --- a/regression-test/data/mtmv_p0/test_iceberg_mtmv.out +++ b/regression-test/data/mtmv_p0/test_iceberg_mtmv.out @@ -118,3 +118,22 @@ 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/suites/mtmv_p0/test_iceberg_mtmv.groovy b/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy index aee80d8d1693a4..36c0d3f120e109 100644 --- a/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy +++ b/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy @@ -69,12 +69,6 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_ // 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 hivePrefix = "hive2"; - String hms_port = context.config.otherConfigs.get(hivePrefix + "HmsPort") - String hdfs_port = context.config.otherConfigs.get(hivePrefix + "HdfsPort") - String default_fs = "hdfs://${externalEnvIp}:${hdfs_port}" - String warehouse = "${default_fs}/warehouse" - String catalog_name = "iceberg_mtmv_catalog_hms"; String mvUnpartition = "test_iceberg_unpartition" String mvName1 = "test_iceberg_mtmv_ts" @@ -85,13 +79,14 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_ String icebergTable2 = "dtable" String icebergTable3 = "union_test" sql """drop catalog if exists ${catalog_name} """ - sql """create catalog if not exists ${catalog_name} properties ( + sql """CREATE CATALOG ${catalog_name} PROPERTIES ( 'type'='iceberg', - 'iceberg.catalog.type'='hms', - 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', - 'fs.defaultFS' = '${default_fs}', - 'warehouse' = '${warehouse}', - 'use_meta_cache' = 'true' + '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""" @@ -266,6 +261,54 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_ 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} """ } } From b671c0128904687bc0e3300495aefc57c9c4c887 Mon Sep 17 00:00:00 2001 From: James Date: Fri, 17 Jan 2025 17:35:21 +0800 Subject: [PATCH 14/27] [mtmv](test)Add more test case for Iceberg mtmv. (#47026) Add more test case for Iceberg mtmv. --- .../iceberg/IcebergExternalTableTest.java | 23 +++++++++++++++++++ .../data/mtmv_p0/test_iceberg_mtmv.out | 11 +++++++++ .../suites/mtmv_p0/test_iceberg_mtmv.groovy | 4 ++++ 3 files changed, 38 insertions(+) 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 index 3ba4804e52279c..75b1d0d9c7e0a0 100644 --- 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 @@ -103,6 +103,29 @@ public Table getIcebergTable() { Assertions.assertTrue(table.isValidRelatedTableCached()); Assertions.assertFalse(table.validRelatedTableCache()); + // Test spec fields are more than 1. + specs.put(0, spec); + table.setIsValidRelatedTableCached(false); + Assertions.assertFalse(table.isValidRelatedTableCached()); + new Expectations() {{ + icebergTable.specs(); + result = specs; + }}; + fields.add(null); + fields.add(null); + new Expectations() {{ + spec.fields(); + result = fields; + }}; + Assertions.assertFalse(table.isValidRelatedTable()); + new Verifications() {{ + spec.fields(); + times = 1; + }}; + Assertions.assertTrue(table.isValidRelatedTableCached()); + Assertions.assertFalse(table.validRelatedTableCache()); + fields.clear(); + // Test true fields.add(field); table.setIsValidRelatedTableCached(false); diff --git a/regression-test/data/mtmv_p0/test_iceberg_mtmv.out b/regression-test/data/mtmv_p0/test_iceberg_mtmv.out index 433cc85d3325eb..47d2d3a61a2eb0 100644 --- a/regression-test/data/mtmv_p0/test_iceberg_mtmv.out +++ b/regression-test/data/mtmv_p0/test_iceberg_mtmv.out @@ -58,6 +58,12 @@ 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 @@ -103,6 +109,11 @@ 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 diff --git a/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy b/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy index 36c0d3f120e109..647009b05d9923 100644 --- a/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy +++ b/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy @@ -138,6 +138,8 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_ waitingMTMVTaskFinishedByMvName(mvName1) 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")) @@ -197,6 +199,8 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_ waitingMTMVTaskFinishedByMvName(mvName2) 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")) From 204dccc1ae7534c0fe4e22038ba77c3cb46fed30 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Fri, 14 Feb 2025 17:22:54 +0800 Subject: [PATCH 15/27] [enhance](mtmv)Add a new class MTMVSnapshotIdSnapshot to make the code more readable (#47166) - before, paimon and iceberg put snapshotId to MTMVVersionSnapshot ,now change to MTMVSnapshotIdSnapshot - `compatiblePartitions` only consider OlapTable, because other TableType not has history data - Delete constructor methods without id in MTMVVersionSnapshot to avoid misuse --- .../iceberg/IcebergExternalTable.java | 6 +- .../mtmv/MTMVRefreshPartitionSnapshot.java | 7 ++- .../doris/mtmv/MTMVSnapshotIdSnapshot.java | 57 +++++++++++++++++++ .../doris/mtmv/MTMVVersionSnapshot.java | 4 -- .../apache/doris/persist/gson/GsonUtils.java | 2 + .../doris/mtmv/MTMVRefreshSnapshotTest.java | 20 +++---- 6 files changed, 78 insertions(+), 18 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVSnapshotIdSnapshot.java 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 2feab480d7edb8..9dd5e5b56705b7 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 @@ -38,8 +38,8 @@ 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.MTMVVersionSnapshot; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ExternalAnalysisTask; @@ -223,7 +223,7 @@ public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshCont if (latestSnapshotId <= 0) { throw new AnalysisException("can not find partition: " + partitionName); } - return new MTMVVersionSnapshot(latestSnapshotId); + return new MTMVSnapshotIdSnapshot(latestSnapshotId); } @Override @@ -231,7 +231,7 @@ public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional 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/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/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/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, From 8783bf2cac4ba8017196f0f0fbb9185b66355d5d Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 5 Mar 2025 17:58:31 +0800 Subject: [PATCH 16/27] [opt](mtmv) metadata cache of external table no longer be refreshed before run async mv task (#48172) Problem Summary: before this PR, external catalog metadata will be sync when refresh async mv that based on external table. after this PR, remove sync metadata action, but the data in async mv still consistent with query in Doris on external table. metadata cache of external table no longer be refreshed before run async mv task --- .../apache/doris/datasource/iceberg/IcebergExternalTable.java | 2 -- 1 file changed, 2 deletions(-) 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 9dd5e5b56705b7..a7547d62328a2a 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 @@ -184,8 +184,6 @@ private IcebergSnapshotCacheValue getIcebergSnapshotCacheValue() { @Override public void beforeMTMVRefresh(MTMV mtmv) throws DdlException { - Env.getCurrentEnv().getRefreshManager() - .refreshTable(getCatalog().getName(), getDbName(), getName(), true); } @Override From d98e5f345e01a5ffd8612952490f39d44369a417 Mon Sep 17 00:00:00 2001 From: Tiewei Fang Date: Wed, 23 Apr 2025 13:52:11 +0800 Subject: [PATCH 17/27] [fix](Hudi-mtmv) Support asynchronous materialized view partition refresh feature for Hudi external tables. (#49956) Problem Summary: Support asynchronous materialized view partition refresh feature for Hudi external tables. --- .../datasource/ExternalMetaCacheMgr.java | 3 +- .../doris/datasource/ExternalTable.java | 2 +- .../datasource/TablePartitionValues.java | 32 +-- .../doris/datasource/hive/HMSDlaTable.java | 79 ++++++ .../datasource/hive/HMSExternalTable.java | 181 ++++++------- .../doris/datasource/hive/HiveDlaTable.java | 141 ++++++++++ .../hive/HiveMetaStoreClientHelper.java | 26 +- .../doris/datasource/hive/HudiDlaTable.java | 123 +++++++++ .../datasource/hive/source/HiveScanNode.java | 3 +- .../datasource/hudi/HudiMvccSnapshot.java | 74 +++++ .../datasource/hudi/HudiSchemaCacheKey.java | 82 ++++++ .../doris/datasource/hudi/HudiUtils.java | 24 +- .../source/HudiCachedPartitionProcessor.java | 30 +- .../datasource/hudi/source/HudiScanNode.java | 65 +++-- .../iceberg/IcebergExternalTable.java | 3 +- .../maxcompute/MaxComputeExternalTable.java | 2 +- .../datasource/mvcc/EmptyMvccSnapshot.java | 21 ++ .../doris/datasource/mvcc/MvccTable.java | 5 +- .../doris/job/extensions/mtmv/MTMVTask.java | 3 +- .../apache/doris/nereids/NereidsPlanner.java | 1 - .../nereids/rules/analysis/BindRelation.java | 1 + .../tablefunction/MetadataGenerator.java | 3 +- .../doris/datasource/hudi/HudiUtilsTest.java | 4 +- .../doris/external/hms/HmsCatalogTest.java | 5 +- .../apache/doris/qe/HmsQueryCacheTest.java | 12 +- .../hudi/hudi_mtmv/test_hudi_mtmv.out | 90 ++++++ .../hudi_mtmv/test_hudi_olap_rewrite_mtmv.out | 44 +++ .../hudi/hudi_mtmv/test_hudi_rewrite_mtmv.out | 16 ++ .../hudi/hudi_mtmv/test_hudi_mtmv.groovy | 256 ++++++++++++++++++ .../test_hudi_olap_rewrite_mtmv.groovy | 108 ++++++++ .../hudi_mtmv/test_hudi_rewrite_mtmv.groovy | 91 +++++++ 31 files changed, 1356 insertions(+), 174 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSDlaTable.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveDlaTable.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HudiDlaTable.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiMvccSnapshot.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiSchemaCacheKey.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/mvcc/EmptyMvccSnapshot.java create mode 100644 regression-test/data/external_table_p2/hudi/hudi_mtmv/test_hudi_mtmv.out create mode 100644 regression-test/data/external_table_p2/hudi/hudi_mtmv/test_hudi_olap_rewrite_mtmv.out create mode 100644 regression-test/data/external_table_p2/hudi/hudi_mtmv/test_hudi_rewrite_mtmv.out create mode 100644 regression-test/suites/external_table_p2/hudi/hudi_mtmv/test_hudi_mtmv.groovy create mode 100644 regression-test/suites/external_table_p2/hudi/hudi_mtmv/test_hudi_olap_rewrite_mtmv.groovy create mode 100644 regression-test/suites/external_table_p2/hudi/hudi_mtmv/test_hudi_rewrite_mtmv.groovy 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 c2f50f929f8a83..3e0c4d90859725 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; @@ -292,7 +293,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..1710646ce3dfbd --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSDlaTable.java @@ -0,0 +1,79 @@ +// 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.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 { + Env.getCurrentEnv().getRefreshManager() + .refreshTable(hmsTable.getCatalog().getName(), hmsTable.getDbName(), hmsTable.getName(), true); + } +} 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 5967bd15b02cf7..ca2d722baf8230 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,21 +31,26 @@ 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.fs.FileSystemDirectoryLister; 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.qe.GlobalVariable; import org.apache.doris.statistics.AnalysisInfo; @@ -78,6 +83,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; @@ -98,7 +106,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; @@ -160,6 +168,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; @@ -197,10 +207,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()); @@ -291,23 +304,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 @@ -347,7 +382,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(); @@ -514,10 +549,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 @@ -526,7 +557,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() { @@ -539,12 +570,12 @@ public long getLastDdlTime() { } @Override - public Optional initSchema() { + public Optional initSchema(SchemaCacheKey key) { makeSureInitialized(); if (dlaType.equals(DLAType.ICEBERG)) { return getIcebergSchema(); } else if (dlaType.equals(DLAType.HUDI)) { - return getHudiSchema(); + return getHudiSchema(key); } else { return getHiveSchema(); } @@ -554,18 +585,26 @@ 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); } @@ -815,96 +854,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(); } /** @@ -966,7 +955,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()) { @@ -1039,6 +1028,16 @@ public boolean isPartitionedTable() { @Override public void beforeMTMVRefresh(MTMV mtmv) throws DdlException { + makeSureInitialized(); + dlaTable.beforeMTMVRefresh(mtmv); + } + + @Override + public MvccSnapshot loadSnapshot(Optional tableSnapshot) { + if (getDlaType() == DLAType.HUDI) { + return new HudiMvccSnapshot(HudiUtils.getPartitionValues(tableSnapshot, this)); + } + return new EmptyMvccSnapshot(); } public boolean firstColumnIsString() { 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..5da80d308fcbd5 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 @@ -61,6 +61,9 @@ 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 +805,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 +818,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 9aff964631b47b..1d71249ed159bf 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/HudiUtils.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java index 0e8d737937ab1c..9387e284d4dcda 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; @@ -251,10 +254,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 +295,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 a6156924e27967..9ade189369d635 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,7 +29,6 @@ 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; @@ -37,6 +36,9 @@ 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.fs.DirectoryLister; import org.apache.doris.planner.PlanNodeId; import org.apache.doris.qe.SessionVariable; import org.apache.doris.spi.Split; @@ -60,10 +62,12 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; +import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.storage.StoragePath; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -72,6 +76,7 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; import java.util.concurrent.Semaphore; @@ -162,11 +167,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 @@ -207,10 +207,18 @@ 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()) - .getFsView(hmsTable.getDbName(), hmsTable.getName(), hudiClient); + .getExtMetaCacheMgr() + .getFsViewProcessor(hmsTable.getCatalog()) + .getFsView(hmsTable.getDbName(), hmsTable.getName(), hudiClient); + if (hudiSchemaCacheValue.isEnableSchemaEvolution()) { + params.setHistorySchemaInfo(new ConcurrentHashMap<>()); + } } @Override @@ -249,18 +257,31 @@ private void setHudiParams(TFileRangeDesc rangeDesc, HudiSplit hudiSplit) { TTableFormatFileDesc tableFormatFileDesc = new TTableFormatFileDesc(); tableFormatFileDesc.setTableFormatType(hudiSplit.getTableFormatType().value()); THudiFileDesc fileDesc = new THudiFileDesc(); - fileDesc.setInstantTime(hudiSplit.getInstantTime()); - fileDesc.setSerde(hudiSplit.getSerde()); - fileDesc.setInputFormat(hudiSplit.getInputFormat()); - fileDesc.setBasePath(hudiSplit.getBasePath()); - fileDesc.setDataFilePath(hudiSplit.getDataFilePath()); - fileDesc.setDataFileLength(hudiSplit.getFileLength()); - fileDesc.setDeltaLogs(hudiSplit.getHudiDeltaLogs()); - fileDesc.setColumnNames(hudiSplit.getHudiColumnNames()); - fileDesc.setColumnTypes(hudiSplit.getHudiColumnTypes()); - // TODO(gaoxin): support complex types - // fileDesc.setNestedFields(hudiSplit.getNestedFields()); - fileDesc.setHudiJniScanner(hudiSplit.getHudiJniScanner()); + if (rangeDesc.getFormatType() == TFileFormatType.FORMAT_JNI) { + fileDesc.setInstantTime(hudiSplit.getInstantTime()); + fileDesc.setSerde(hudiSplit.getSerde()); + fileDesc.setInputFormat(hudiSplit.getInputFormat()); + fileDesc.setBasePath(hudiSplit.getBasePath()); + fileDesc.setDataFilePath(hudiSplit.getDataFilePath()); + fileDesc.setDataFileLength(hudiSplit.getFileLength()); + fileDesc.setDeltaLogs(hudiSplit.getHudiDeltaLogs()); + fileDesc.setColumnNames(hudiSplit.getHudiColumnNames()); + fileDesc.setColumnTypes(hudiSplit.getHudiColumnTypes()); + // TODO(gaoxin): support complex types + // fileDesc.setNestedFields(hudiSplit.getNestedFields()); + } else { + HudiSchemaCacheValue hudiSchemaCacheValue = HudiUtils.getSchemaCacheValue(hmsTable, queryInstant); + if (hudiSchemaCacheValue.isEnableSchemaEvolution()) { + long commitInstantTime = Long.parseLong(FSUtils.getCommitTime( + new File(hudiSplit.getPath().get()).getName())); + InternalSchema internalSchema = hudiSchemaCacheValue + .getCommitInstantInternalSchema(hudiClient, commitInstantTime); + params.history_schema_info.computeIfAbsent( + internalSchema.schemaId(), + k -> HudiUtils.getSchemaInfo(internalSchema)); + fileDesc.setSchemaId(internalSchema.schemaId()); //for schema change. (native reader) + } + } tableFormatFileDesc.setHudiParams(fileDesc); rangeDesc.setTableFormatParams(tableFormatFileDesc); } @@ -270,7 +291,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 a7547d62328a2a..1f54946d8bf40c 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 @@ -18,6 +18,7 @@ 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; @@ -278,7 +279,7 @@ public boolean isValidRelatedTable() { } @Override - public MvccSnapshot loadSnapshot() { + public MvccSnapshot loadSnapshot(Optional tableSnapshot) { return new IcebergMvccSnapshot(getIcebergSnapshotCacheValue()); } 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/job/extensions/mtmv/MTMVTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java index c74091c11d4ba2..b456da97ab12cd 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 @@ -83,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; @@ -423,7 +424,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/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java index 85e40c6c7b37fb..c1a31ecf30c343 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 @@ -220,7 +220,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/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index c1d7a81e28a77d..586972e649509d 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/tablefunction/MetadataGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java index 1420e03f375c22..34ec59b1919182 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java @@ -62,6 +62,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; @@ -1607,7 +1608,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/external/hms/HmsCatalogTest.java b/fe/fe-core/src/test/java/org/apache/doris/external/hms/HmsCatalogTest.java index 30a233dd1a999e..4abed2ee708b95 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)); new Expectations(tbl) { { tbl.getId(); @@ -138,7 +141,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(); 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 0a981dab8a9dcc..376f8cba4e8c24 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.nereids.datasets.tpch.AnalyzeCheckTestBase; import org.apache.doris.planner.OlapScanNode; @@ -123,6 +125,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)); new Expectations(tbl) { { tbl.getId(); @@ -158,7 +161,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(); @@ -173,6 +176,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)); new Expectations(tbl2) { { tbl2.getId(); @@ -208,7 +212,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(); @@ -386,7 +390,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); @@ -434,7 +438,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/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}""" +} From e21a1ec0309ca250e3e47e11a56b5db3dd54b63e Mon Sep 17 00:00:00 2001 From: Tiewei Fang Date: Thu, 22 May 2025 09:47:08 +0800 Subject: [PATCH 18/27] [Fix](MTMV) mtmv not refresh cache of external table before run task (#50979) Problem Summary: related pr: #48172 This pr(#48172) had changed the code logical of method `beforeMTMVRefresh`, but this pr(#49956) added the code back. So we delete this code. --- .../org/apache/doris/datasource/hive/HMSDlaTable.java | 3 --- .../apache/doris/datasource/hive/HMSExternalTable.java | 2 -- .../data/mtmv_p0/test_hive_refresh_mtmv.out | 6 ++++++ .../suites/mtmv_p0/test_hive_refresh_mtmv.groovy | 10 +++++++++- 4 files changed, 15 insertions(+), 6 deletions(-) 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 index 1710646ce3dfbd..7894279b2950ac 100644 --- 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 @@ -18,7 +18,6 @@ package org.apache.doris.datasource.hive; 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; @@ -73,7 +72,5 @@ abstract MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional Date: Fri, 20 Jun 2025 11:57:36 +0800 Subject: [PATCH 19/27] fix error --- .../datasource/hive/HMSExternalTable.java | 4 +- .../datasource/hudi/HudiSchemaCacheValue.java | 17 +- .../doris/datasource/hudi/HudiUtils.java | 4 + .../datasource/iceberg/IcebergUtils.java | 533 +----------------- .../paimon/PaimonExternalTable.java | 65 ++- .../paimon/PaimonMetadataCache.java | 48 +- .../paimon/PaimonPartitionInfo.java | 11 +- .../paimon/PaimonSchemaCacheValue.java | 12 +- .../doris/datasource/paimon/PaimonUtil.java | 112 +++- .../apache/doris/mtmv/MTMVPartitionUtil.java | 1 + .../doris/nereids/StatementContext.java | 5 +- 11 files changed, 198 insertions(+), 614 deletions(-) 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 49520cc67357d3..f113fc29c8848f 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 @@ -46,12 +46,12 @@ 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.fs.FileSystemDirectoryLister; import org.apache.doris.mtmv.MTMVBaseTableIf; import org.apache.doris.mtmv.MTMVRefreshContext; import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.mtmv.MTMVSnapshotIf; import org.apache.doris.nereids.exceptions.NotSupportedException; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan.SelectedPartitions; import org.apache.doris.qe.GlobalVariable; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; @@ -573,7 +573,7 @@ public long getLastDdlTime() { public Optional initSchema(SchemaCacheKey key) { makeSureInitialized(); if (dlaType.equals(DLAType.ICEBERG)) { - return getIcebergSchema(); + return getIcebergSchema(key); } else if (dlaType.equals(DLAType.HUDI)) { return getHudiSchema(key); } else { 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 9387e284d4dcda..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 @@ -45,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; 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 716d12f6a13d7b..48560d3d886911 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 @@ -31,96 +31,56 @@ import org.apache.doris.analysis.LiteralExpr; import org.apache.doris.analysis.NullLiteral; import org.apache.doris.analysis.PartitionDesc; -import org.apache.doris.analysis.PartitionValue; import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.StringLiteral; import org.apache.doris.analysis.Subquery; -import org.apache.doris.analysis.TableScanParams; -import org.apache.doris.analysis.TableSnapshot; import org.apache.doris.catalog.ArrayType; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.MapType; -import org.apache.doris.catalog.PartitionItem; -import org.apache.doris.catalog.PartitionKey; -import org.apache.doris.catalog.RangePartitionItem; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.StructField; import org.apache.doris.catalog.StructType; import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.Type; -import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; import org.apache.doris.common.info.SimpleTableInfo; import org.apache.doris.common.util.TimeUtils; -import org.apache.doris.datasource.CacheException; import org.apache.doris.datasource.ExternalCatalog; -import org.apache.doris.datasource.ExternalSchemaCache; import org.apache.doris.datasource.SchemaCacheValue; -import org.apache.doris.datasource.iceberg.source.IcebergTableQueryInfo; -import org.apache.doris.datasource.mvcc.MvccSnapshot; -import org.apache.doris.datasource.mvcc.MvccUtil; +import org.apache.doris.datasource.hive.HiveMetaStoreClientHelper; 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.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.commons.lang3.exception.ExceptionUtils; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.ManifestFile; -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.Schema; import org.apache.iceberg.Snapshot; -import org.apache.iceberg.SnapshotRef; -import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.expressions.And; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.expressions.ManifestEvaluator; import org.apache.iceberg.expressions.Not; import org.apache.iceberg.expressions.Or; -import org.apache.iceberg.expressions.Projections; import org.apache.iceberg.expressions.Unbound; import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.types.Type.TypeID; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.LocationUtil; -import org.apache.iceberg.util.SnapshotUtil; -import org.apache.iceberg.util.StructProjection; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.io.IOException; -import java.time.DateTimeException; -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.List; import java.util.Locale; import java.util.Map; import java.util.Optional; -import java.util.Set; -import java.util.regex.Pattern; import java.util.stream.Collectors; /** @@ -151,17 +111,7 @@ 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; // means an empty table - public static final long NEWEST_SCHEMA_ID = -1; - - 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 static final Pattern SNAPSHOT_ID = Pattern.compile("\\d+"); + public static final long UNKNOWN_SNAPSHOT_ID = -1; public static Expression convertToIcebergExpr(Expr expr, Schema schema) { if (expr == null) { @@ -630,30 +580,25 @@ private static org.apache.iceberg.Table getIcebergTableInternal(ExternalCatalog * Get iceberg schema from catalog and convert them to doris schema */ 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; - if (schemaId == NEWEST_SCHEMA_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) { - tmpSchema.add(new Column(field.name().toLowerCase(Locale.ROOT), - IcebergUtils.icebergTypeToDorisType(field.type()), true, null, true, field.doc(), true, - schema.caseInsensitiveFindField(field.name()).fieldId())); - } - return tmpSchema; - }); - } catch (Exception e) { - throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e), e); - } - + return HiveMetaStoreClientHelper.ugiDoAs(catalog.getConfiguration(), () -> { + org.apache.iceberg.Table icebergTable = getIcebergTable(catalog, dbName, name); + 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) { + tmpSchema.add(new Column(field.name().toLowerCase(Locale.ROOT), + IcebergUtils.icebergTypeToDorisType(field.type()), true, null, true, field.doc(), true, + schema.caseInsensitiveFindField(field.name()).fieldId())); + } + return tmpSchema; + }); } @@ -746,12 +691,6 @@ public static HiveCatalog createIcebergHiveCatalog(ExternalCatalog externalCatal hiveCatalog.setConf(externalCatalog.getConfiguration()); Map catalogProperties = externalCatalog.getProperties(); - if (!catalogProperties.containsKey(HiveCatalog.LIST_ALL_TABLES)) { - // This configuration will display all tables (including non-Iceberg type tables), - // which can save the time of obtaining table objects. - // Later, type checks will be performed when loading the table. - catalogProperties.put(HiveCatalog.LIST_ALL_TABLES, "true"); - } String metastoreUris = catalogProperties.getOrDefault(HMSProperties.HIVE_METASTORE_URIS, ""); catalogProperties.put(CatalogProperties.URI, metastoreUris); @@ -759,410 +698,6 @@ public static HiveCatalog createIcebergHiveCatalog(ExternalCatalog externalCatal return hiveCatalog; } - // Retrieve the manifest files that match the query based on partitions in filter - public static CloseableIterable getMatchingManifest( - List dataManifests, - Map specsById, - Expression dataFilter) { - LoadingCache evalCache = Caffeine.newBuilder() - .build( - specId -> { - PartitionSpec spec = specsById.get(specId); - return ManifestEvaluator.forPartitionFilter( - Expressions.and( - Expressions.alwaysTrue(), - Projections.inclusive(spec, true).project(dataFilter)), - spec, - true); - }); - - CloseableIterable matchingManifests = CloseableIterable.filter( - CloseableIterable.withNoopClose(dataManifests), - manifest -> evalCache.get(manifest.partitionSpecId()).eval(manifest)); - - matchingManifests = - CloseableIterable.filter( - matchingManifests, - manifest -> manifest.hasAddedFiles() || manifest.hasExistingFiles()); - - return matchingManifests; - } - - // get snapshot id from query like 'for version/time as of' or '@branch/@tag' - public static IcebergTableQueryInfo getQuerySpecSnapshot( - Table table, - Optional queryTableSnapshot, - Optional scanParams) throws UserException { - - Preconditions.checkArgument( - queryTableSnapshot.isPresent() || isIcebergBranchOrTag(scanParams), - "should spec version or time or branch or tag"); - - // not support `select * from tb@branch/tag(b) for version/time as of ...` - Preconditions.checkArgument( - !(queryTableSnapshot.isPresent() && isIcebergBranchOrTag(scanParams)), - "could not spec a version/time with tag/branch"); - - // solve @branch/@tag - if (scanParams.isPresent()) { - String refName; - TableScanParams params = scanParams.get(); - if (!params.getMapParams().isEmpty()) { - refName = params.getMapParams().get("name"); - } else { - refName = params.getListParams().get(0); - } - SnapshotRef snapshotRef = table.refs().get(refName); - if (params.isBranch()) { - if (snapshotRef == null || !snapshotRef.isBranch()) { - throw new UserException("Table " + table.name() + " does not have branch named " + refName); - } - } else { - if (snapshotRef == null || !snapshotRef.isTag()) { - throw new UserException("Table " + table.name() + " does not have tag named " + refName); - } - } - return new IcebergTableQueryInfo( - snapshotRef.snapshotId(), - refName, - SnapshotUtil.schemaFor(table, refName).schemaId()); - } - - // solve version/time as of - String value = queryTableSnapshot.get().getValue(); - TableSnapshot.VersionType type = queryTableSnapshot.get().getType(); - if (type == TableSnapshot.VersionType.VERSION) { - if (SNAPSHOT_ID.matcher(value).matches()) { - long snapshotId = Long.parseLong(value); - Snapshot snapshot = table.snapshot(snapshotId); - if (snapshot == null) { - throw new UserException("Table " + table.name() + " does not have snapshotId " + value); - } - return new IcebergTableQueryInfo( - snapshotId, - null, - snapshot.schemaId() - ); - } - - if (!table.refs().containsKey(value)) { - throw new UserException("Table " + table.name() + " does not have tag or branch named " + value); - } - return new IcebergTableQueryInfo( - table.refs().get(value).snapshotId(), - value, - SnapshotUtil.schemaFor(table, value).schemaId() - ); - } else { - long timestamp = TimeUtils.timeStringToLong(value, TimeUtils.getTimeZone()); - if (timestamp < 0) { - throw new DateTimeException("can't parse time: " + value); - } - long snapshotId = SnapshotUtil.snapshotIdAsOfTime(table, timestamp); - return new IcebergTableQueryInfo( - snapshotId, - null, - table.snapshot(snapshotId).schemaId() - ); - } - } - - public static boolean isIcebergBranchOrTag(Optional scanParams) { - if (scanParams == null || !scanParams.isPresent()) { - return false; - } - TableScanParams params = scanParams.get(); - if (params.isBranch() || params.isTag()) { - if (!params.getMapParams().isEmpty()) { - Preconditions.checkArgument( - params.getMapParams().containsKey("name"), - "must contain key 'name' in params" - ); - } else { - Preconditions.checkArgument( - params.getListParams().size() == 1 - && params.getListParams().get(0) != null, - "must contain a branch/tag name in params" - ); - } - return true; - } - return false; - } - - // read schema from external schema cache - public static IcebergSchemaCacheValue getSchemaCacheValue( - ExternalCatalog catalog, String dbName, String name, 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 static IcebergSnapshot getLastedIcebergSnapshot(ExternalCatalog catalog, String dbName, String tbName) { - Table table = IcebergUtils.getIcebergTable(catalog, dbName, tbName); - Snapshot snapshot = table.currentSnapshot(); - long snapshotId = snapshot == null ? IcebergUtils.UNKNOWN_SNAPSHOT_ID : snapshot.snapshotId(); - return new IcebergSnapshot(snapshotId, table.schema().schemaId()); - } - - public static IcebergPartitionInfo loadPartitionInfo( - ExternalCatalog catalog, String dbName, String tbName, long snapshotId) throws AnalysisException { - // snapshotId == UNKNOWN_SNAPSHOT_ID means this is an empty table, haven't contained any snapshot yet. - if (snapshotId == IcebergUtils.UNKNOWN_SNAPSHOT_ID) { - return IcebergPartitionInfo.empty(); - } - Table table = getIcebergTable(catalog, dbName, tbName); - List icebergPartitions = loadIcebergPartition(table, snapshotId); - Map nameToPartition = Maps.newHashMap(); - Map nameToPartitionItem = Maps.newHashMap(); - - List partitionColumns = IcebergUtils.getSchemaCacheValue( - catalog, dbName, tbName, 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); - } - - private static List loadIcebergPartition(Table table, 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(table, row)); - } - } - } catch (IOException e) { - LOG.warn("Failed to get Iceberg table {} partition info.", table.name(), e); - } - return partitions; - } - - private static IcebergPartition generateIcebergPartition(Table table, 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 - 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 static 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. - */ - @VisibleForTesting - public static 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) - */ - @VisibleForTesting - public static 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; - } - } - - public static IcebergSnapshotCacheValue getIcebergSnapshotCacheValue( - Optional tableSnapshot, - ExternalCatalog catalog, - String dbName, - String tbName, - Optional scanParams) { - IcebergSnapshotCacheValue snapshotCache = Env.getCurrentEnv().getExtMetaCacheMgr().getIcebergMetadataCache() - .getSnapshotCache(catalog, dbName, tbName); - if (tableSnapshot.isPresent() || IcebergUtils.isIcebergBranchOrTag(scanParams)) { - // If a snapshot is specified, - // use the specified snapshot and the corresponding schema(not the latest schema). - Table icebergTable = getIcebergTable(catalog, dbName, tbName); - IcebergTableQueryInfo info; - try { - info = getQuerySpecSnapshot(icebergTable, tableSnapshot, scanParams); - } catch (UserException e) { - throw new RuntimeException(e); - } - return new IcebergSnapshotCacheValue( - IcebergPartitionInfo.empty(), - new IcebergSnapshot(info.getSnapshotId(), info.getSchemaId())); - } else { - // Otherwise, use the latest snapshot and the latest schema. - return snapshotCache; - } - } - // load table schema from iceberg API to external schema cache. public static Optional loadSchemaCacheValue( ExternalCatalog catalog, String dbName, String tbName, long schemaId) { @@ -1181,30 +716,4 @@ public static Optional loadSchemaCacheValue( } return Optional.of(new IcebergSchemaCacheValue(schema, tmpColumns)); } - - public static List getIcebergSchema( - TableIf tableIf, - ExternalCatalog catalog, - String dbName, - String tbName) { - Optional snapshotFromContext = MvccUtil.getSnapshotFromContext(tableIf); - IcebergSnapshotCacheValue cacheValue = - IcebergUtils.getOrFetchSnapshotCacheValue(snapshotFromContext, catalog, dbName, tbName); - return IcebergUtils.getSchemaCacheValue( - catalog, dbName, tbName, cacheValue.getSnapshot().getSchemaId()) - .getSchema(); - } - - public static IcebergSnapshotCacheValue getOrFetchSnapshotCacheValue( - Optional snapshot, - ExternalCatalog catalog, - String dbName, - String tbName) { - if (snapshot.isPresent()) { - return ((IcebergMvccSnapshot) snapshot.get()).getSnapshotCacheValue(); - } else { - return IcebergUtils.getIcebergSnapshotCacheValue( - Optional.empty(), catalog, dbName, tbName, Optional.empty()); - } - } } 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 635f3038bcf9a6..ef953a0836d813 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,7 +17,6 @@ package org.apache.doris.datasource.paimon; -import org.apache.doris.analysis.TableScanParams; import org.apache.doris.analysis.TableSnapshot; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; @@ -34,8 +33,6 @@ 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; @@ -54,13 +51,17 @@ import com.google.common.collect.Sets; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.paimon.partition.Partition; -import org.apache.paimon.schema.TableSchema; -import org.apache.paimon.table.DataTable; +import org.apache.paimon.CoreOptions; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.predicate.PredicateBuilder; import org.apache.paimon.table.Table; import org.apache.paimon.table.source.Split; +import org.apache.paimon.table.system.SchemasTable; import org.apache.paimon.types.DataField; +import java.io.IOException; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -93,7 +94,9 @@ protected synchronized void makeSureInitialized() { } public Table getPaimonTable(Optional snapshot) { - return getOrFetchSnapshotCacheValue(snapshot).getSnapshot().getTable(); + return paimonTable.copy( + Collections.singletonMap(CoreOptions.SCAN_VERSION.key(), + String.valueOf(getOrFetchSnapshotCacheValue(snapshot).getSnapshot().getSnapshotId()))); } public PaimonSchemaCacheValue getPaimonSchemaCacheValue(long schemaId) { @@ -190,32 +193,21 @@ private boolean isPartitionInvalid(Optional snapshot) { public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, Optional snapshot) throws AnalysisException { - Partition paimonPartition = getOrFetchSnapshotCacheValue(snapshot).getPartitionInfo().getNameToPartition() + PaimonPartition paimonPartition = getOrFetchSnapshotCacheValue(snapshot).getPartitionInfo().getNameToPartition() .get(partitionName); if (paimonPartition == null) { throw new AnalysisException("can not find partition: " + partitionName); } - return new MTMVTimestampSnapshot(paimonPartition.lastFileCreationTime()); + return new MTMVTimestampSnapshot(paimonPartition.getLastUpdateTime()); } @Override public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) throws AnalysisException { - return getTableSnapshot(snapshot); - } - - @Override - public MTMVSnapshotIf getTableSnapshot(Optional snapshot) throws AnalysisException { PaimonSnapshotCacheValue paimonSnapshot = getOrFetchSnapshotCacheValue(snapshot); return new MTMVSnapshotIdSnapshot(paimonSnapshot.getSnapshot().getSnapshotId()); } - @Override - public long getNewestUpdateVersionOrTime() { - return getPaimonSnapshotCacheValue().getPartitionInfo().getNameToPartition().values().stream() - .mapToLong(Partition::lastFileCreationTime).max().orElse(0); - } - @Override public boolean isPartitionColumnAllowNull() { // Paimon will write to the 'null' partition regardless of whether it is' null or 'null'. @@ -227,7 +219,7 @@ public boolean isPartitionColumnAllowNull() { } @Override - public MvccSnapshot loadSnapshot(Optional tableSnapshot, Optional scanParams) { + public MvccSnapshot loadSnapshot(Optional tableSnapshot) { return new PaimonMvccSnapshot(getPaimonSnapshotCacheValue()); } @@ -251,11 +243,10 @@ public Optional initSchema(SchemaCacheKey key) { makeSureInitialized(); PaimonSchemaCacheKey paimonSchemaCacheKey = (PaimonSchemaCacheKey) key; try { - Table table = ((PaimonExternalCatalog) getCatalog()).getPaimonTable(key.getDbName(), name); - TableSchema tableSchema = ((DataTable) table).schemaManager().schema(paimonSchemaCacheKey.getSchemaId()); - List columns = tableSchema.fields(); + PaimonSchema schema = loadPaimonSchemaBySchemaId(paimonSchemaCacheKey); + List columns = schema.getFields(); List dorisColumns = Lists.newArrayListWithCapacity(columns.size()); - Set partitionColumnNames = Sets.newHashSet(tableSchema.partitionKeys()); + Set partitionColumnNames = Sets.newHashSet(schema.getPartitionKeys()); List partitionColumns = Lists.newArrayList(); for (DataField field : columns) { Column column = new Column(field.name().toLowerCase(), @@ -266,7 +257,7 @@ public Optional initSchema(SchemaCacheKey key) { partitionColumns.add(column); } } - return Optional.of(new PaimonSchemaCacheValue(dorisColumns, partitionColumns, tableSchema)); + return Optional.of(new PaimonSchemaCacheValue(dorisColumns, partitionColumns)); } catch (Exception e) { throw new CacheException("failed to initSchema for: %s.%s.%s.%s", null, getCatalog().getName(), key.getDbName(), key.getTblName(), @@ -275,6 +266,23 @@ null, getCatalog().getName(), key.getDbName(), key.getTblName(), } + private PaimonSchema loadPaimonSchemaBySchemaId(PaimonSchemaCacheKey key) throws IOException { + Table table = ((PaimonExternalCatalog) getCatalog()).getPaimonTable(key.getDbName(), + name + Catalog.SYSTEM_TABLE_SPLITTER + SchemasTable.SCHEMAS); + PredicateBuilder builder = new PredicateBuilder(table.rowType()); + Predicate predicate = builder.equal(0, key.getSchemaId()); + // Adding predicates will also return excess data + List rows = PaimonUtil.read(table, new int[] {0, 1, 2}, predicate); + for (InternalRow row : rows) { + PaimonSchema schema = PaimonUtil.rowToSchema(row); + if (schema.getSchemaId() == key.getSchemaId()) { + return schema; + } + } + throw new CacheException("failed to initSchema for: %s.%s.%s.%s", + null, getCatalog().getName(), key.getDbName(), key.getTblName(), key.getSchemaId()); + } + private PaimonSchemaCacheValue getPaimonSchemaCacheValue(Optional snapshot) { PaimonSnapshotCacheValue snapshotCacheValue = getOrFetchSnapshotCacheValue(snapshot); return getPaimonSchemaCacheValue(snapshotCacheValue.getSnapshot().getSchemaId()); @@ -288,9 +296,4 @@ private PaimonSnapshotCacheValue getOrFetchSnapshotCacheValue(Optional getSupportedSysTables() { - makeSureInitialized(); - return SupportedSysTables.PAIMON_SUPPORTED_SYS_TABLES; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCache.java index dfc1e6a1c25c61..109394fabded6a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonMetadataCache.java @@ -26,19 +26,19 @@ import org.apache.doris.datasource.ExternalMetaCacheMgr; import com.github.benmanes.caffeine.cache.LoadingCache; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.commons.collections.CollectionUtils; -import org.apache.paimon.CoreOptions; -import org.apache.paimon.Snapshot; -import org.apache.paimon.partition.Partition; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.data.InternalRow; import org.apache.paimon.table.Table; +import org.apache.paimon.table.system.PartitionsTable; +import org.apache.paimon.table.system.SnapshotsTable; import org.jetbrains.annotations.NotNull; import java.io.IOException; -import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.OptionalLong; import java.util.concurrent.ExecutorService; @@ -75,27 +75,39 @@ private PaimonSnapshotCacheValue loadSnapshot(PaimonSnapshotCacheKey key) { private PaimonPartitionInfo loadPartitionInfo(PaimonSnapshotCacheKey key, List partitionColumns) throws IOException, AnalysisException { if (CollectionUtils.isEmpty(partitionColumns)) { - return PaimonPartitionInfo.EMPTY; + return new PaimonPartitionInfo(); } - List paimonPartitions = ((PaimonExternalCatalog) key.getCatalog()) - .getPaimonPartitions(key.getDbName(), key.getTableName()); + List paimonPartitions = loadPartitions(key); return PaimonUtil.generatePartitionInfo(partitionColumns, paimonPartitions); } + private List loadPartitions(PaimonSnapshotCacheKey key) + throws IOException { + Table table = ((PaimonExternalCatalog) key.getCatalog()).getPaimonTable(key.getDbName(), + key.getTableName() + Catalog.SYSTEM_TABLE_SPLITTER + PartitionsTable.PARTITIONS); + List rows = PaimonUtil.read(table, null, null); + List res = Lists.newArrayListWithCapacity(rows.size()); + for (InternalRow row : rows) { + res.add(PaimonUtil.rowToPartition(row)); + } + return res; + } + private PaimonSnapshot loadLatestSnapshot(PaimonSnapshotCacheKey key) throws IOException { - Table table = ((PaimonExternalCatalog) key.getCatalog()).getPaimonTable(key.getDbName(), key.getTableName()); - Table snapshotTable = table; + Table table = ((PaimonExternalCatalog) key.getCatalog()).getPaimonTable(key.getDbName(), + key.getTableName() + Catalog.SYSTEM_TABLE_SPLITTER + SnapshotsTable.SNAPSHOTS); // snapshotId and schemaId - Long latestSnapshotId = PaimonSnapshot.INVALID_SNAPSHOT_ID; + List rows = PaimonUtil.read(table, new int[] {0, 1}, null); + long latestSnapshotId = 0L; long latestSchemaId = 0L; - Optional optionalSnapshot = table.latestSnapshot(); - if (optionalSnapshot.isPresent()) { - latestSnapshotId = optionalSnapshot.get().id(); - latestSchemaId = table.snapshot(latestSnapshotId).schemaId(); - snapshotTable = - table.copy(Collections.singletonMap(CoreOptions.SCAN_SNAPSHOT_ID.key(), latestSnapshotId.toString())); + for (InternalRow row : rows) { + long snapshotId = row.getLong(0); + if (snapshotId > latestSnapshotId) { + latestSnapshotId = snapshotId; + latestSchemaId = row.getLong(1); + } } - return new PaimonSnapshot(latestSnapshotId, latestSchemaId, snapshotTable); + return new PaimonSnapshot(latestSnapshotId, latestSchemaId); } public void invalidateCatalogCache(long catalogId) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonPartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonPartitionInfo.java index a6339ef5155e15..88515a2510d2c7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonPartitionInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonPartitionInfo.java @@ -20,23 +20,20 @@ import org.apache.doris.catalog.PartitionItem; import com.google.common.collect.Maps; -import org.apache.paimon.partition.Partition; import java.util.Map; public class PaimonPartitionInfo { - public static final PaimonPartitionInfo EMPTY = new PaimonPartitionInfo(); - private final Map nameToPartitionItem; - private final Map nameToPartition; + private final Map nameToPartition; - private PaimonPartitionInfo() { + public PaimonPartitionInfo() { this.nameToPartitionItem = Maps.newHashMap(); this.nameToPartition = Maps.newHashMap(); } public PaimonPartitionInfo(Map nameToPartitionItem, - Map nameToPartition) { + Map nameToPartition) { this.nameToPartitionItem = nameToPartitionItem; this.nameToPartition = nameToPartition; } @@ -45,7 +42,7 @@ public Map getNameToPartitionItem() { return nameToPartitionItem; } - public Map getNameToPartition() { + public Map getNameToPartition() { return nameToPartition; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheValue.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheValue.java index e931b52336ba8f..ccb530a3cbccc7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheValue.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonSchemaCacheValue.java @@ -20,28 +20,18 @@ import org.apache.doris.catalog.Column; import org.apache.doris.datasource.SchemaCacheValue; -import org.apache.paimon.schema.TableSchema; - import java.util.List; public class PaimonSchemaCacheValue extends SchemaCacheValue { private List partitionColumns; - private TableSchema tableSchema; - // Caching TableSchema can reduce the reading of schema files and json parsing. - - public PaimonSchemaCacheValue(List schema, List partitionColumns, TableSchema tableSchema) { + public PaimonSchemaCacheValue(List schema, List partitionColumns) { super(schema); this.partitionColumns = partitionColumns; - this.tableSchema = tableSchema; } public List getPartitionColumns() { return partitionColumns; } - - public TableSchema getTableSchema() { - return tableSchema; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtil.java index caf4eecaa3a470..bbb1eaf5096520 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtil.java @@ -36,18 +36,17 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.serializer.InternalRowSerializer; import org.apache.paimon.options.ConfigOption; -import org.apache.paimon.partition.Partition; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.type.TypeReference; import org.apache.paimon.table.Table; import org.apache.paimon.table.source.ReadBuilder; import org.apache.paimon.types.ArrayType; -import org.apache.paimon.types.CharType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DecimalType; import org.apache.paimon.types.MapType; import org.apache.paimon.types.RowType; -import org.apache.paimon.types.VarCharType; +import org.apache.paimon.utils.JsonSerdeUtil; import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.Projection; @@ -70,9 +69,7 @@ public static List read( for (Pair, String> pair : dynamicOptions) { options.put(pair.getKey().key(), pair.getValue()); } - if (!options.isEmpty()) { - table = table.copy(options); - } + table = table.copy(options); ReadBuilder readBuilder = table.newReadBuilder(); if (projection != null) { readBuilder.withProjection(projection); @@ -92,40 +89,71 @@ public static List read( return rows; } - public static PaimonPartitionInfo generatePartitionInfo(List partitionColumns, - List paimonPartitions) { - if (CollectionUtils.isEmpty(partitionColumns) || paimonPartitions.isEmpty()) { - return PaimonPartitionInfo.EMPTY; - } + /* + https://paimon.apache.org/docs/0.9/maintenance/system-tables/#partitions-table + +---------------+----------------+--------------------+--------------------+------------------------+ + | partition | record_count | file_size_in_bytes| file_count| last_update_time| + +---------------+----------------+--------------------+--------------------+------------------------+ + | [1] | 1 | 645 | 1 | 2024-06-24 10:25:57.400| + +---------------+----------------+--------------------+--------------------+------------------------+ + org.apache.paimon.table.system.PartitionsTable.TABLE_TYPE + public static final RowType TABLE_TYPE = + new RowType( + Arrays.asList( + new DataField(0, "partition", SerializationUtils.newStringType(true)), + new DataField(1, "record_count", new BigIntType(false)), + new DataField(2, "file_size_in_bytes", new BigIntType(false)), + new DataField(3, "file_count", new BigIntType(false)), + new DataField(4, "last_update_time", DataTypes.TIMESTAMP_MILLIS()))); + */ + public static PaimonPartition rowToPartition(InternalRow row) { + String partition = row.getString(0).toString(); + long recordCount = row.getLong(1); + long fileSizeInBytes = row.getLong(2); + long fileCount = row.getLong(3); + long lastUpdateTime = row.getTimestamp(4, 3).getMillisecond(); + return new PaimonPartition(partition, recordCount, fileSizeInBytes, fileCount, lastUpdateTime); + } + public static PaimonPartitionInfo generatePartitionInfo(List partitionColumns, + List paimonPartitions) { Map nameToPartitionItem = Maps.newHashMap(); - Map nameToPartition = Maps.newHashMap(); + Map nameToPartition = Maps.newHashMap(); PaimonPartitionInfo partitionInfo = new PaimonPartitionInfo(nameToPartitionItem, nameToPartition); - - for (Partition partition : paimonPartitions) { - Map spec = partition.spec(); - StringBuilder sb = new StringBuilder(); - for (Map.Entry entry : spec.entrySet()) { - sb.append(entry.getKey()).append("=").append(entry.getValue()).append("/"); - } - if (sb.length() > 0) { - sb.deleteCharAt(sb.length() - 1); - } - String partitionName = sb.toString(); - nameToPartition.put(partitionName, partition); + if (CollectionUtils.isEmpty(partitionColumns)) { + return partitionInfo; + } + for (PaimonPartition paimonPartition : paimonPartitions) { + String partitionName = getPartitionName(partitionColumns, paimonPartition.getPartitionValues()); + nameToPartition.put(partitionName, paimonPartition); try { // partition values return by paimon api, may have problem, // to avoid affecting the query, we catch exceptions here nameToPartitionItem.put(partitionName, toListPartitionItem(partitionName, partitionColumns)); } catch (Exception e) { - LOG.warn("toListPartitionItem failed, partitionColumns: {}, partitionValues: {}", - partitionColumns, partition.spec(), e); + LOG.warn("toListPartitionItem failed, partitionColumns: {}, partitionValues: {}", partitionColumns, + paimonPartition.getPartitionValues(), e); } } return partitionInfo; } + private static String getPartitionName(List partitionColumns, String partitionValueStr) { + Preconditions.checkNotNull(partitionValueStr); + String[] partitionValues = partitionValueStr.replace("[", "").replace("]", "") + .split(","); + Preconditions.checkState(partitionColumns.size() == partitionValues.length); + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < partitionColumns.size(); ++i) { + if (i != 0) { + sb.append("/"); + } + sb.append(partitionColumns.get(i).getName()).append("=").append(partitionValues[i]); + } + return sb.toString(); + } + public static ListPartitionItem toListPartitionItem(String partitionName, List partitionColumns) throws AnalysisException { List types = partitionColumns.stream() @@ -167,10 +195,8 @@ private static Type paimonPrimitiveTypeToDorisType(org.apache.paimon.types.DataT case TINYINT: return Type.TINYINT; case VARCHAR: - return ScalarType.createVarcharType(((VarCharType) dataType).getLength()); - case CHAR: - return ScalarType.createCharType(((CharType) dataType).getLength()); case BINARY: + case CHAR: case VARBINARY: return Type.STRING; case DECIMAL: @@ -225,4 +251,32 @@ private static Type paimonPrimitiveTypeToDorisType(org.apache.paimon.types.DataT public static Type paimonTypeToDorisType(org.apache.paimon.types.DataType type) { return paimonPrimitiveTypeToDorisType(type); } + + /** + * https://paimon.apache.org/docs/0.9/maintenance/system-tables/#schemas-table + * demo: + * 0 + * [{"id":0,"name":"user_id","type":"BIGINT NOT NULL"}, + * {"id":1,"name":"item_id","type":"BIGINT"}, + * {"id":2,"name":"behavior","type":"STRING"}, + * {"id":3,"name":"dt","type":"STRING NOT NULL"}, + * {"id":4,"name":"hh","type":"STRING NOT NULL"}] + * ["dt"] + * ["dt","hh","user_id"] + * {"owner":"hadoop","provider":"paimon"} + * 2024-12-03 15:38:14.734 + * + * @param row + * @return + */ + public static PaimonSchema rowToSchema(InternalRow row) { + long schemaId = row.getLong(0); + String fieldsStr = row.getString(1).toString(); + String partitionKeysStr = row.getString(2).toString(); + List fields = JsonSerdeUtil.fromJson(fieldsStr, new TypeReference>() { + }); + List partitionKeys = JsonSerdeUtil.fromJson(partitionKeysStr, new TypeReference>() { + }); + return new PaimonSchema(schemaId, fields, partitionKeys); + } } 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 47f61357c21f02..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 @@ -53,6 +53,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.regex.Matcher; import java.util.regex.Pattern; 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 670d235b9a8539..fc4b77eb4ec802 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,7 +18,6 @@ package org.apache.doris.nereids; import org.apache.doris.analysis.StatementBase; -import org.apache.doris.analysis.TableScanParams; import org.apache.doris.analysis.TableSnapshot; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.TableIf; @@ -648,13 +647,13 @@ public void addPlannerHook(PlannerHook plannerHook) { /** * Load snapshot information of mvcc */ - public void loadSnapshots(Optional tableSnapshot, Optional scanParams) { + 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(tableSnapshot, scanParams)); + snapshots.put(mvccTableInfo, ((MvccTable) tableIf).loadSnapshot(tableSnapshot)); } } } From 9af68d7ab2709c58fb5de5fb7b80f799b3a2cb1a Mon Sep 17 00:00:00 2001 From: zhangdong Date: Fri, 20 Jun 2025 12:27:20 +0800 Subject: [PATCH 20/27] success --- .../datasource/hudi/source/HudiScanNode.java | 51 ++++++------------- .../iceberg/IcebergExternalTableTest.java | 6 +-- 2 files changed, 18 insertions(+), 39 deletions(-) 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 9ade189369d635..489717c85d3c7d 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 @@ -31,14 +31,12 @@ import org.apache.doris.common.util.LocationPath; 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.fs.DirectoryLister; import org.apache.doris.planner.PlanNodeId; import org.apache.doris.qe.SessionVariable; import org.apache.doris.spi.Split; @@ -62,12 +60,10 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; -import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.storage.StoragePath; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -76,7 +72,6 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; import java.util.concurrent.Semaphore; @@ -213,12 +208,9 @@ protected void doInitialize() throws UserException { columnTypes = hudiSchemaCacheValue.getColTypes(); fsView = Env.getCurrentEnv() - .getExtMetaCacheMgr() - .getFsViewProcessor(hmsTable.getCatalog()) - .getFsView(hmsTable.getDbName(), hmsTable.getName(), hudiClient); - if (hudiSchemaCacheValue.isEnableSchemaEvolution()) { - params.setHistorySchemaInfo(new ConcurrentHashMap<>()); - } + .getExtMetaCacheMgr() + .getFsViewProcessor(hmsTable.getCatalog()) + .getFsView(hmsTable.getDbName(), hmsTable.getName(), hudiClient); } @Override @@ -257,31 +249,18 @@ private void setHudiParams(TFileRangeDesc rangeDesc, HudiSplit hudiSplit) { TTableFormatFileDesc tableFormatFileDesc = new TTableFormatFileDesc(); tableFormatFileDesc.setTableFormatType(hudiSplit.getTableFormatType().value()); THudiFileDesc fileDesc = new THudiFileDesc(); - if (rangeDesc.getFormatType() == TFileFormatType.FORMAT_JNI) { - fileDesc.setInstantTime(hudiSplit.getInstantTime()); - fileDesc.setSerde(hudiSplit.getSerde()); - fileDesc.setInputFormat(hudiSplit.getInputFormat()); - fileDesc.setBasePath(hudiSplit.getBasePath()); - fileDesc.setDataFilePath(hudiSplit.getDataFilePath()); - fileDesc.setDataFileLength(hudiSplit.getFileLength()); - fileDesc.setDeltaLogs(hudiSplit.getHudiDeltaLogs()); - fileDesc.setColumnNames(hudiSplit.getHudiColumnNames()); - fileDesc.setColumnTypes(hudiSplit.getHudiColumnTypes()); - // TODO(gaoxin): support complex types - // fileDesc.setNestedFields(hudiSplit.getNestedFields()); - } else { - HudiSchemaCacheValue hudiSchemaCacheValue = HudiUtils.getSchemaCacheValue(hmsTable, queryInstant); - if (hudiSchemaCacheValue.isEnableSchemaEvolution()) { - long commitInstantTime = Long.parseLong(FSUtils.getCommitTime( - new File(hudiSplit.getPath().get()).getName())); - InternalSchema internalSchema = hudiSchemaCacheValue - .getCommitInstantInternalSchema(hudiClient, commitInstantTime); - params.history_schema_info.computeIfAbsent( - internalSchema.schemaId(), - k -> HudiUtils.getSchemaInfo(internalSchema)); - fileDesc.setSchemaId(internalSchema.schemaId()); //for schema change. (native reader) - } - } + fileDesc.setInstantTime(hudiSplit.getInstantTime()); + fileDesc.setSerde(hudiSplit.getSerde()); + fileDesc.setInputFormat(hudiSplit.getInputFormat()); + fileDesc.setBasePath(hudiSplit.getBasePath()); + fileDesc.setDataFilePath(hudiSplit.getDataFilePath()); + fileDesc.setDataFileLength(hudiSplit.getFileLength()); + fileDesc.setDeltaLogs(hudiSplit.getHudiDeltaLogs()); + fileDesc.setColumnNames(hudiSplit.getHudiColumnNames()); + fileDesc.setColumnTypes(hudiSplit.getHudiColumnTypes()); + // TODO(gaoxin): support complex types + // fileDesc.setNestedFields(hudiSplit.getNestedFields()); + fileDesc.setHudiJniScanner(hudiSplit.getHudiJniScanner()); tableFormatFileDesc.setHudiParams(fileDesc); rangeDesc.setTableFormatParams(tableFormatFileDesc); } 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 index 75b1d0d9c7e0a0..c7d50643decc70 100644 --- 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 @@ -54,7 +54,7 @@ public void testIsSupportedPartitionTable(@Mocked org.apache.iceberg.Table icebe @Mocked PartitionSpec spec, @Mocked PartitionField field, @Mocked Schema schema) { - IcebergExternalTable table = new IcebergExternalTable(1, "1", "2", null); + IcebergExternalTable table = new IcebergExternalTable(1, "1", "2", null, null); Map specs = Maps.newHashMap(); new MockUp() { @Mock @@ -170,7 +170,7 @@ public Table getIcebergTable() { @Test public void testGetPartitionRange() throws AnalysisException { - IcebergExternalTable table = new IcebergExternalTable(1, "1", "2", null); + IcebergExternalTable table = new IcebergExternalTable(1, "1", "2", null, null); Column c = new Column("ts", PrimitiveType.DATETIMEV2); List partitionColumns = Lists.newArrayList(c); table.setPartitionColumns(partitionColumns); @@ -219,7 +219,7 @@ public void testGetPartitionRange() throws AnalysisException { @Test public void testSortRange() throws AnalysisException { - IcebergExternalTable table = new IcebergExternalTable(1, "1", "2", null); + IcebergExternalTable table = new IcebergExternalTable(1, "1", "2", null, null); Column c = new Column("c", PrimitiveType.DATETIMEV2); ArrayList columns = Lists.newArrayList(c); table.setPartitionColumns(Lists.newArrayList(c)); From 8f9a02576e5c8ae0ab1495ca14c67be87eeeebdd Mon Sep 17 00:00:00 2001 From: zhangdong Date: Fri, 20 Jun 2025 14:16:09 +0800 Subject: [PATCH 21/27] up IcebergUtils to 47782 --- .../datasource/iceberg/IcebergUtils.java | 55 ++++++++++++------- 1 file changed, 35 insertions(+), 20 deletions(-) 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 48560d3d886911..f9d0bc84d63c9f 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 @@ -48,13 +48,13 @@ import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.SchemaCacheValue; -import org.apache.doris.datasource.hive.HiveMetaStoreClientHelper; import org.apache.doris.datasource.property.constants.HMSProperties; import org.apache.doris.nereids.exceptions.NotSupportedException; import org.apache.doris.thrift.TExprOpcode; 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.PartitionField; @@ -576,29 +576,38 @@ 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, long schemaId) { - return HiveMetaStoreClientHelper.ugiDoAs(catalog.getConfiguration(), () -> { - org.apache.iceberg.Table icebergTable = getIcebergTable(catalog, dbName, name); - 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) { - tmpSchema.add(new Column(field.name().toLowerCase(Locale.ROOT), - IcebergUtils.icebergTypeToDorisType(field.type()), true, null, true, field.doc(), true, - schema.caseInsensitiveFindField(field.name()).fieldId())); - } - return tmpSchema; - }); + try { + return catalog.getPreExecutionAuthenticator().execute(() -> { + org.apache.iceberg.Table icebergTable = getIcebergTable(catalog, dbName, name); + 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) { + tmpSchema.add(new Column(field.name().toLowerCase(Locale.ROOT), + IcebergUtils.icebergTypeToDorisType(field.type()), true, null, true, field.doc(), true, + schema.caseInsensitiveFindField(field.name()).fieldId())); + } + return tmpSchema; + }); + } catch (Exception e) { + throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e), e); + } + } @@ -691,6 +700,12 @@ public static HiveCatalog createIcebergHiveCatalog(ExternalCatalog externalCatal hiveCatalog.setConf(externalCatalog.getConfiguration()); Map catalogProperties = externalCatalog.getProperties(); + if (!catalogProperties.containsKey(HiveCatalog.LIST_ALL_TABLES)) { + // This configuration will display all tables (including non-Iceberg type tables), + // which can save the time of obtaining table objects. + // Later, type checks will be performed when loading the table. + catalogProperties.put(HiveCatalog.LIST_ALL_TABLES, "true"); + } String metastoreUris = catalogProperties.getOrDefault(HMSProperties.HIVE_METASTORE_URIS, ""); catalogProperties.put(CatalogProperties.URI, metastoreUris); From 913d4a76a008c7aaec65ee2a857987ce9063dfae Mon Sep 17 00:00:00 2001 From: zhangdong Date: Fri, 20 Jun 2025 14:23:01 +0800 Subject: [PATCH 22/27] format --- .../apache/doris/datasource/hive/HiveMetaStoreClientHelper.java | 2 -- .../src/main/java/org/apache/doris/nereids/CascadesContext.java | 1 - 2 files changed, 3 deletions(-) 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 5da80d308fcbd5..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,7 +57,6 @@ 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; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index 866734c6cf3c7d..1486f03e269b13 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -56,7 +56,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; -import org.apache.commons.collections.MapUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; From 039436e241aa123c2c737e88cdec274328904ee6 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Thu, 9 Jan 2025 16:32:25 +0800 Subject: [PATCH 23/27] [enhance](mtmv)When obtaining the partition list fails, treat the paimon table as an unpartitioned table (#46641) When retrieving data of type Paimon Date in version 0.9 from the system table, the value is an integer and cannot be converted to type Date. This issue has been fixed in Paimon's latest code. This PR downgrades this situation without affecting user data queries --- .../data/mtmv_p0/test_paimon_mtmv.out | 6 ++++ .../suites/mtmv_p0/test_paimon_mtmv.groovy | 30 +++++++++++++++++++ 2 files changed, 36 insertions(+) diff --git a/regression-test/data/mtmv_p0/test_paimon_mtmv.out b/regression-test/data/mtmv_p0/test_paimon_mtmv.out index ba6fc06c1d2491..0cfe4bd293cf51 100644 --- a/regression-test/data/mtmv_p0/test_paimon_mtmv.out +++ b/regression-test/data/mtmv_p0/test_paimon_mtmv.out @@ -140,3 +140,9 @@ true 4 null 5 NULL +-- !date_partition_base_table -- +1 2020-01-01 + +-- !date_partition -- +1 2020-01-01 + diff --git a/regression-test/suites/mtmv_p0/test_paimon_mtmv.groovy b/regression-test/suites/mtmv_p0/test_paimon_mtmv.groovy index 48d63e03ec3db5..98477ab92d542e 100644 --- a/regression-test/suites/mtmv_p0/test_paimon_mtmv.groovy +++ b/regression-test/suites/mtmv_p0/test_paimon_mtmv.groovy @@ -264,6 +264,36 @@ suite("test_paimon_mtmv", "p0,external,mtmv,external_docker,external_docker_dori 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}""" } From 2f0af605f77b2db009daa9d8c7c9b93b3f558ebe Mon Sep 17 00:00:00 2001 From: zhangdong Date: Fri, 20 Jun 2025 18:35:15 +0800 Subject: [PATCH 24/27] 1 --- .../iceberg/IcebergExternalTableTest.java | 256 +----------------- 1 file changed, 1 insertion(+), 255 deletions(-) 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 index c7d50643decc70..1032d2ce58f221 100644 --- 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 @@ -17,261 +17,7 @@ package org.apache.doris.datasource.iceberg; -import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.PartitionItem; -import org.apache.doris.catalog.PartitionKey; -import org.apache.doris.catalog.PrimitiveType; -import org.apache.doris.catalog.RangePartitionItem; -import org.apache.doris.common.AnalysisException; - -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Range; -import mockit.Expectations; -import mockit.Mock; -import mockit.MockUp; -import mockit.Mocked; -import mockit.Verifications; -import org.apache.iceberg.PartitionField; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.transforms.Days; -import org.apache.iceberg.transforms.Hours; -import org.apache.iceberg.transforms.Months; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Set; - public class IcebergExternalTableTest { - @Test - public void testIsSupportedPartitionTable(@Mocked org.apache.iceberg.Table icebergTable, - @Mocked PartitionSpec spec, - @Mocked PartitionField field, - @Mocked Schema schema) { - IcebergExternalTable table = new IcebergExternalTable(1, "1", "2", null, null); - Map specs = Maps.newHashMap(); - new MockUp() { - @Mock - private void makeSureInitialized() { - } - - @Mock - public Table getIcebergTable() { - return icebergTable; - } - }; - // Test null - specs.put(0, null); - new Expectations() {{ - icebergTable.specs(); - result = specs; - }}; - table.setTable(icebergTable); - Assertions.assertFalse(table.isValidRelatedTableCached()); - Assertions.assertFalse(table.isValidRelatedTable()); - new Verifications() {{ - icebergTable.specs(); - times = 1; - }}; - Assertions.assertTrue(table.isValidRelatedTableCached()); - Assertions.assertFalse(table.validRelatedTableCache()); - - // Test spec fields are empty. - specs.put(0, spec); - table.setIsValidRelatedTableCached(false); - Assertions.assertFalse(table.isValidRelatedTableCached()); - new Expectations() {{ - icebergTable.specs(); - result = specs; - }}; - List fields = Lists.newArrayList(); - new Expectations() {{ - spec.fields(); - result = fields; - }}; - Assertions.assertFalse(table.isValidRelatedTable()); - new Verifications() {{ - spec.fields(); - times = 1; - }}; - Assertions.assertTrue(table.isValidRelatedTableCached()); - Assertions.assertFalse(table.validRelatedTableCache()); - - // Test spec fields are more than 1. - specs.put(0, spec); - table.setIsValidRelatedTableCached(false); - Assertions.assertFalse(table.isValidRelatedTableCached()); - new Expectations() {{ - icebergTable.specs(); - result = specs; - }}; - fields.add(null); - fields.add(null); - new Expectations() {{ - spec.fields(); - result = fields; - }}; - Assertions.assertFalse(table.isValidRelatedTable()); - new Verifications() {{ - spec.fields(); - times = 1; - }}; - Assertions.assertTrue(table.isValidRelatedTableCached()); - Assertions.assertFalse(table.validRelatedTableCache()); - fields.clear(); - - // Test true - fields.add(field); - table.setIsValidRelatedTableCached(false); - Assertions.assertFalse(table.isValidRelatedTableCached()); - new Expectations() { - { - icebergTable.schema(); - result = schema; - - schema.findColumnName(anyInt); - result = "col1"; - } - }; - new Expectations() {{ - field.transform(); - result = new Hours(); - }}; - Assertions.assertTrue(table.isValidRelatedTable()); - Assertions.assertTrue(table.isValidRelatedTableCached()); - Assertions.assertTrue(table.validRelatedTableCache()); - new Verifications() {{ - schema.findColumnName(anyInt); - times = 1; - }}; - new Expectations() {{ - field.transform(); - result = new Days(); - }}; - table.setIsValidRelatedTableCached(false); - Assertions.assertFalse(table.isValidRelatedTableCached()); - Assertions.assertTrue(table.isValidRelatedTable()); - new Expectations() {{ - field.transform(); - result = new Months(); - }}; - table.setIsValidRelatedTableCached(false); - Assertions.assertFalse(table.isValidRelatedTableCached()); - Assertions.assertTrue(table.isValidRelatedTable()); - Assertions.assertTrue(table.isValidRelatedTableCached()); - Assertions.assertTrue(table.validRelatedTableCache()); - } - - @Test - public void testGetPartitionRange() throws AnalysisException { - IcebergExternalTable table = new IcebergExternalTable(1, "1", "2", null, null); - Column c = new Column("ts", PrimitiveType.DATETIMEV2); - List partitionColumns = Lists.newArrayList(c); - table.setPartitionColumns(partitionColumns); - - // Test null partition value - Range nullRange = table.getPartitionRange(null, "hour", partitionColumns); - Assertions.assertEquals("0000-01-01 00:00:00", - nullRange.lowerEndpoint().getPartitionValuesAsStringList().get(0)); - Assertions.assertEquals("0000-01-01 00:00:01", - nullRange.upperEndpoint().getPartitionValuesAsStringList().get(0)); - - // Test hour transform. - Range hour = table.getPartitionRange("100", "hour", partitionColumns); - PartitionKey lowKey = hour.lowerEndpoint(); - PartitionKey upKey = hour.upperEndpoint(); - Assertions.assertEquals("1970-01-05 04:00:00", lowKey.getPartitionValuesAsStringList().get(0)); - Assertions.assertEquals("1970-01-05 05:00:00", upKey.getPartitionValuesAsStringList().get(0)); - - // Test day transform. - Range day = table.getPartitionRange("100", "day", partitionColumns); - lowKey = day.lowerEndpoint(); - upKey = day.upperEndpoint(); - Assertions.assertEquals("1970-04-11 00:00:00", lowKey.getPartitionValuesAsStringList().get(0)); - Assertions.assertEquals("1970-04-12 00:00:00", upKey.getPartitionValuesAsStringList().get(0)); - - // Test month transform. - Range month = table.getPartitionRange("100", "month", partitionColumns); - lowKey = month.lowerEndpoint(); - upKey = month.upperEndpoint(); - Assertions.assertEquals("1978-05-01 00:00:00", lowKey.getPartitionValuesAsStringList().get(0)); - Assertions.assertEquals("1978-06-01 00:00:00", upKey.getPartitionValuesAsStringList().get(0)); - - // Test year transform. - Range year = table.getPartitionRange("100", "year", partitionColumns); - lowKey = year.lowerEndpoint(); - upKey = year.upperEndpoint(); - Assertions.assertEquals("2070-01-01 00:00:00", lowKey.getPartitionValuesAsStringList().get(0)); - Assertions.assertEquals("2071-01-01 00:00:00", upKey.getPartitionValuesAsStringList().get(0)); - - // Test unsupported transform - Exception exception = Assertions.assertThrows(RuntimeException.class, () -> { - table.getPartitionRange("100", "bucket", partitionColumns); - }); - Assertions.assertEquals("Unsupported transform bucket", exception.getMessage()); - } - - @Test - public void testSortRange() throws AnalysisException { - IcebergExternalTable table = new IcebergExternalTable(1, "1", "2", null, null); - Column c = new Column("c", PrimitiveType.DATETIMEV2); - ArrayList columns = Lists.newArrayList(c); - table.setPartitionColumns(Lists.newArrayList(c)); - PartitionItem nullRange = new RangePartitionItem(table.getPartitionRange(null, "hour", columns)); - PartitionItem year1970 = new RangePartitionItem(table.getPartitionRange("0", "year", columns)); - PartitionItem year1971 = new RangePartitionItem(table.getPartitionRange("1", "year", columns)); - PartitionItem month197002 = new RangePartitionItem(table.getPartitionRange("1", "month", columns)); - PartitionItem month197103 = new RangePartitionItem(table.getPartitionRange("14", "month", columns)); - PartitionItem month197204 = new RangePartitionItem(table.getPartitionRange("27", "month", columns)); - PartitionItem day19700202 = new RangePartitionItem(table.getPartitionRange("32", "day", columns)); - PartitionItem day19730101 = new RangePartitionItem(table.getPartitionRange("1096", "day", columns)); - Map map = Maps.newHashMap(); - map.put("nullRange", nullRange); - map.put("year1970", year1970); - map.put("year1971", year1971); - map.put("month197002", month197002); - map.put("month197103", month197103); - map.put("month197204", month197204); - map.put("day19700202", day19700202); - map.put("day19730101", day19730101); - List> entries = table.sortPartitionMap(map); - Assertions.assertEquals(8, entries.size()); - Assertions.assertEquals("nullRange", entries.get(0).getKey()); - Assertions.assertEquals("year1970", entries.get(1).getKey()); - Assertions.assertEquals("month197002", entries.get(2).getKey()); - Assertions.assertEquals("day19700202", entries.get(3).getKey()); - Assertions.assertEquals("year1971", entries.get(4).getKey()); - Assertions.assertEquals("month197103", entries.get(5).getKey()); - Assertions.assertEquals("month197204", entries.get(6).getKey()); - Assertions.assertEquals("day19730101", entries.get(7).getKey()); - - Map> stringSetMap = table.mergeOverlapPartitions(map); - Assertions.assertEquals(2, stringSetMap.size()); - Assertions.assertTrue(stringSetMap.containsKey("year1970")); - Assertions.assertTrue(stringSetMap.containsKey("year1971")); - - Set names1970 = stringSetMap.get("year1970"); - Assertions.assertEquals(3, names1970.size()); - Assertions.assertTrue(names1970.contains("year1970")); - Assertions.assertTrue(names1970.contains("month197002")); - Assertions.assertTrue(names1970.contains("day19700202")); - - Set names1971 = stringSetMap.get("year1971"); - Assertions.assertEquals(2, names1971.size()); - Assertions.assertTrue(names1971.contains("year1971")); - Assertions.assertTrue(names1971.contains("month197103")); - - Assertions.assertEquals(5, map.size()); - Assertions.assertTrue(map.containsKey("nullRange")); - Assertions.assertTrue(map.containsKey("year1970")); - Assertions.assertTrue(map.containsKey("year1971")); - Assertions.assertTrue(map.containsKey("month197204")); - Assertions.assertTrue(map.containsKey("day19730101")); - } + // branch3.1 need pick by other pr } From ef1d5896102a9eeb12c07bbcaca573c2a320b970 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Mon, 23 Jun 2025 10:44:23 +0800 Subject: [PATCH 25/27] resolve conflict --- .../apache/doris/datasource/iceberg/IcebergExternalTable.java | 4 ++-- .../org/apache/doris/datasource/iceberg/IcebergUtils.java | 4 ++-- .../apache/doris/datasource/paimon/PaimonExternalTable.java | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) 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 2be998b1f51a0e..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 @@ -36,13 +36,13 @@ 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.datasource.systable.SupportedSysTables; -import org.apache.doris.datasource.systable.SysTable; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ExternalAnalysisTask; 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 a5e89413ca87cc..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 @@ -52,15 +52,15 @@ import org.apache.doris.nereids.exceptions.NotSupportedException; import org.apache.doris.thrift.TExprOpcode; -import com.google.common.base.Preconditions; 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.PartitionField; 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; 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 d2eb02fff7242a..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 @@ -33,14 +33,14 @@ 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.mtmv.MTMVTimestampSnapshot; -import org.apache.doris.datasource.systable.SupportedSysTables; -import org.apache.doris.datasource.systable.SysTable; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ExternalAnalysisTask; From f0ad32466ecd66faeb1d25d14c096a4bab7295f0 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Mon, 23 Jun 2025 11:00:55 +0800 Subject: [PATCH 26/27] resolve conflict --- .../apache/doris/external/hms/HmsCatalogTest.java | 8 ++++---- .../java/org/apache/doris/qe/HmsQueryCacheTest.java | 12 ++++++++---- 2 files changed, 12 insertions(+), 8 deletions(-) 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/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); From e87e6114c179f9afd7aeed9a92323b26c3da0467 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Mon, 23 Jun 2025 11:33:23 +0800 Subject: [PATCH 27/27] fix case --- .../suites/mtmv_p0/test_iceberg_mtmv.groovy | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy b/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy index 647009b05d9923..30b9e8e9f7329c 100644 --- a/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy +++ b/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy @@ -107,35 +107,35 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_ 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) + 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) + 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) + waitingMTMVTaskFinishedByMvName(mvName1, dbName) qt_test_ts_refresh3 """select * from ${mvName1} order by value""" sql """REFRESH MATERIALIZED VIEW ${mvName1} auto""" - waitingMTMVTaskFinishedByMvName(mvName1) + 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) + waitingMTMVTaskFinishedByMvName(mvName1, dbName) qt_test_ts_refresh5 """select * from ${mvName1} order by value""" sql """REFRESH MATERIALIZED VIEW ${mvName1} auto""" - waitingMTMVTaskFinishedByMvName(mvName1) + 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) + 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};""" @@ -178,25 +178,25 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_ 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) + 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) + 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) + waitingMTMVTaskFinishedByMvName(mvName2, dbName) qt_test_d_refresh3 "select * from ${mvName2} order by value" sql """REFRESH MATERIALIZED VIEW ${mvName2} partitions(p_20240901_20241001);""" - waitingMTMVTaskFinishedByMvName(mvName2) + 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) + 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};""" @@ -244,7 +244,7 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_ // refresh one partiton sql """REFRESH MATERIALIZED VIEW ${mvName} partitions(p_20240101000000_20240102000000);""" - waitingMTMVTaskFinishedByMvName(mvName) + waitingMTMVTaskFinishedByMvName(mvName, dbName) order_qt_refresh_one_partition "SELECT * FROM ${mvName} " def explainOnePartition = sql """ explain ${mvSql} """ logger.info("explainOnePartition: " + explainOnePartition.toString()) @@ -254,7 +254,7 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_ //refresh auto sql """REFRESH MATERIALIZED VIEW ${mvName} auto""" - waitingMTMVTaskFinishedByMvName(mvName) + waitingMTMVTaskFinishedByMvName(mvName, dbName) order_qt_refresh_auto "SELECT * FROM ${mvName} " def explainAllPartition = sql """ explain ${mvSql}; """ logger.info("explainAllPartition: " + explainAllPartition.toString())