From 3aa6d044df16bf93570a6586c91e3fb986406aee Mon Sep 17 00:00:00 2001 From: Gabriel Date: Thu, 21 Nov 2024 10:18:58 +0800 Subject: [PATCH 001/110] [refactor](column) Simplify column functions (#44272) `insert_many_binary_data` has the same logics as `insert_many_strings`. This PR unify those 2 functions. --- .../rowset/segment_v2/binary_plain_page.h | 19 ++++++--------- .../rowset/segment_v2/segment_iterator.cpp | 6 +++-- be/src/vec/columns/column.h | 6 ----- be/src/vec/columns/column_complex.h | 5 ++-- .../vec/columns/column_fixed_length_object.h | 7 ++---- be/src/vec/columns/column_nullable.cpp | 11 ++++++--- be/src/vec/columns/column_nullable.h | 6 ----- be/src/vec/columns/column_object.h | 6 ----- be/src/vec/columns/column_string.h | 23 ------------------- be/src/vec/columns/predicate_column.h | 19 +++++++-------- 10 files changed, 31 insertions(+), 77 deletions(-) diff --git a/be/src/olap/rowset/segment_v2/binary_plain_page.h b/be/src/olap/rowset/segment_v2/binary_plain_page.h index e043164ef286337..3b3c6ad3feab923 100644 --- a/be/src/olap/rowset/segment_v2/binary_plain_page.h +++ b/be/src/olap/rowset/segment_v2/binary_plain_page.h @@ -266,8 +266,7 @@ class BinaryPlainPageDecoder : public PageDecoder { auto total = *n; size_t read_count = 0; - _len_array.resize(total); - _start_offset_array.resize(total); + _binary_data.resize(total); for (size_t i = 0; i < total; ++i) { ordinal_t ord = rowids[i] - page_first_ordinal; if (UNLIKELY(ord >= _num_elems)) { @@ -275,14 +274,13 @@ class BinaryPlainPageDecoder : public PageDecoder { } const uint32_t start_offset = offset(ord); - _start_offset_array[read_count] = start_offset; - _len_array[read_count] = offset(ord + 1) - start_offset; + _binary_data[read_count].data = _data.mutable_data() + start_offset; + _binary_data[read_count].size = offset(ord + 1) - start_offset; read_count++; } if (LIKELY(read_count > 0)) { - dst->insert_many_binary_data(_data.mutable_data(), _len_array.data(), - _start_offset_array.data(), read_count); + dst->insert_many_strings(_binary_data.data(), read_count); } *n = read_count; @@ -342,13 +340,11 @@ class BinaryPlainPageDecoder : public PageDecoder { if (idx >= _num_elems) { return _offsets_pos; } - const uint8_t* p = - reinterpret_cast(&_data[_offsets_pos + idx * SIZE_OF_INT32]); - return decode_fixed32_le(p); + return guarded_offset(idx); } uint32_t guarded_offset(size_t idx) const { - const uint8_t* p = + const auto* p = reinterpret_cast(&_data[_offsets_pos + idx * SIZE_OF_INT32]); return decode_fixed32_le(p); } @@ -361,8 +357,7 @@ class BinaryPlainPageDecoder : public PageDecoder { uint32_t _offsets_pos; std::vector _offsets; - std::vector _len_array; - std::vector _start_offset_array; + std::vector _binary_data; // Index of the currently seeked element in the page. uint32_t _cur_idx; diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp index 2c7aa5a0ce1cd05..96b0bea2ae82125 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -2264,8 +2264,10 @@ Status SegmentIterator::_next_batch_internal(vectorized::Block* block) { size_t rows = block->rows(); for (const auto& entry : *block) { if (entry.column->size() != rows) { - throw doris::Exception(ErrorCode::INTERNAL_ERROR, "unmatched size {}, expected {}", - entry.column->size(), rows); + throw doris::Exception(ErrorCode::INTERNAL_ERROR, + "unmatched size {}, expected {}, column: {}, type: {}", + entry.column->size(), rows, entry.column->get_name(), + entry.type->get_name()); } } #endif diff --git a/be/src/vec/columns/column.h b/be/src/vec/columns/column.h index ce155aefad23210..e92c246741f2e4d 100644 --- a/be/src/vec/columns/column.h +++ b/be/src/vec/columns/column.h @@ -249,12 +249,6 @@ class IColumn : public COW { "Method insert_many_dict_data is not supported for " + get_name()); } - virtual void insert_many_binary_data(char* data_array, uint32_t* len_array, - uint32_t* start_offset_array, size_t num) { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "Method insert_many_binary_data is not supported for " + get_name()); - } - /// Insert binary data into column from a continuous buffer, the implementation maybe copy all binary data /// in one single time. virtual void insert_many_continuous_binary_data(const char* data, const uint32_t* offsets, diff --git a/be/src/vec/columns/column_complex.h b/be/src/vec/columns/column_complex.h index 1ff074b112977fc..24b6b7ddbd7b549 100644 --- a/be/src/vec/columns/column_complex.h +++ b/be/src/vec/columns/column_complex.h @@ -98,10 +98,9 @@ class ColumnComplexType final : public COWHelper> } } - void insert_many_binary_data(char* data_array, uint32_t* len_array, - uint32_t* start_offset_array, size_t num) override { + void insert_many_strings(const StringRef* strings, size_t num) override { for (size_t i = 0; i < num; i++) { - insert_binary_data(data_array + start_offset_array[i], len_array[i]); + insert_binary_data(strings[i].data, strings[i].size); } } diff --git a/be/src/vec/columns/column_fixed_length_object.h b/be/src/vec/columns/column_fixed_length_object.h index ea3437dbb011cec..947d859ce231d08 100644 --- a/be/src/vec/columns/column_fixed_length_object.h +++ b/be/src/vec/columns/column_fixed_length_object.h @@ -284,8 +284,7 @@ class ColumnFixedLengthObject final : public COWHelperinsert_default(); + _push_false_to_nullmap(not_null_count); + not_null_count = 0; get_null_map_data().push_back(1); _has_null = true; } else { - nested_column->insert_data(strings[i].data, strings[i].size); - _push_false_to_nullmap(1); + not_null_count++; } } + if (not_null_count) { + _push_false_to_nullmap(not_null_count); + } + nested_column->insert_many_strings(strings, num); } void ColumnNullable::insert_many_from(const IColumn& src, size_t position, size_t length) { diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h index 0e5e104dce9a8dd..2b87aa982cae501 100644 --- a/be/src/vec/columns/column_nullable.h +++ b/be/src/vec/columns/column_nullable.h @@ -245,12 +245,6 @@ class ColumnNullable final : public COWHelper, public N get_nested_column().insert_many_continuous_binary_data(data, offsets, num); } - void insert_many_binary_data(char* data_array, uint32_t* len_array, - uint32_t* start_offset_array, size_t num) override { - _push_false_to_nullmap(num); - get_nested_column().insert_many_binary_data(data_array, len_array, start_offset_array, num); - } - void insert_default() override { get_nested_column().insert_default(); get_null_map_data().push_back(1); diff --git a/be/src/vec/columns/column_object.h b/be/src/vec/columns/column_object.h index 244692e9ddd1bc9..1c8f38056c9d546 100644 --- a/be/src/vec/columns/column_object.h +++ b/be/src/vec/columns/column_object.h @@ -465,12 +465,6 @@ class ColumnObject final : public COWHelper { "insert_many_dict_data" + get_name()); } - void insert_many_binary_data(char* data_array, uint32_t* len_array, - uint32_t* start_offset_array, size_t num) override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "insert_many_binary_data" + get_name()); - } - void insert_many_continuous_binary_data(const char* data, const uint32_t* offsets, const size_t num) override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, diff --git a/be/src/vec/columns/column_string.h b/be/src/vec/columns/column_string.h index a72a3ec5cdc96a4..f116d4ce1f17cfa 100644 --- a/be/src/vec/columns/column_string.h +++ b/be/src/vec/columns/column_string.h @@ -269,29 +269,6 @@ class ColumnStr final : public COWHelper> { DCHECK(chars.size() == offsets.back()); } - void insert_many_binary_data(char* data_array, uint32_t* len_array, - uint32_t* start_offset_array, size_t num) override { - size_t new_size = 0; - for (size_t i = 0; i < num; i++) { - new_size += len_array[i]; - } - - const size_t old_size = chars.size(); - check_chars_length(old_size + new_size, offsets.size() + num); - chars.resize(old_size + new_size); - - Char* data = chars.data(); - size_t offset = old_size; - for (size_t i = 0; i < num; i++) { - uint32_t len = len_array[i]; - uint32_t start_offset = start_offset_array[i]; - // memcpy will deal len == 0, not do it here - memcpy(data + offset, data_array + start_offset, len); - offset += len; - offsets.push_back(offset); - } - } - void insert_many_strings(const StringRef* strings, size_t num) override { size_t new_size = 0; for (size_t i = 0; i < num; i++) { diff --git a/be/src/vec/columns/predicate_column.h b/be/src/vec/columns/predicate_column.h index c9ceec230f7aec8..c2c6456d86255bb 100644 --- a/be/src/vec/columns/predicate_column.h +++ b/be/src/vec/columns/predicate_column.h @@ -269,8 +269,7 @@ class PredicateColumnType final : public COWHelperalloc(total_mem_size); char* org_dst = destination; size_t org_elem_num = data.size(); data.resize(org_elem_num + num); - uint32_t fragment_start_offset = start_offset_array[0]; + uint32_t fragment_start_offset = 0; size_t fragment_len = 0; for (size_t i = 0; i < num; i++) { data[org_elem_num + i].data = destination + fragment_len; - data[org_elem_num + i].size = len_array[i]; - fragment_len += len_array[i]; + data[org_elem_num + i].size = strings[i].size; + fragment_len += strings[i].size; // Compute the largest continuous memcpy block and copy them. // If this is the last element in data array, then should copy the current memory block. - if (i == num - 1 || - start_offset_array[i + 1] != start_offset_array[i] + len_array[i]) { - memcpy(destination, data_array + fragment_start_offset, fragment_len); + if (i == num - 1 || strings[i + 1].data != strings[i].data + strings[i].size) { + memcpy(destination, strings[fragment_start_offset].data, fragment_len); destination += fragment_len; - fragment_start_offset = (i == num - 1 ? 0 : start_offset_array[i + 1]); + fragment_start_offset = i == num - 1 ? 0 : i + 1; fragment_len = 0; } } @@ -309,7 +307,6 @@ class PredicateColumnType final : public COWHelper Date: Thu, 21 Nov 2024 10:45:19 +0800 Subject: [PATCH 002/110] [feat](mtmv)mtmv support paimon partition refresh (#43959) ### What problem does this PR solve? 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 ### Release note mtmv support paimon partition refresh --- .../paimon/PaimonExternalTable.java | 159 +++++++++++++++++- .../datasource/paimon/PaimonPartition.java | 61 +++++++ .../paimon/PaimonPartitionInfo.java | 48 ++++++ .../paimon/PaimonSchemaCacheValue.java | 23 ++- .../doris/datasource/paimon/PaimonUtil.java | 155 +++++++++++++++++ .../org/apache/doris/mtmv/PaimonUtilTest.java | 71 ++++++++ .../data/mtmv_p0/test_paimon_mtmv.out | 116 ++++++++++++- .../suites/mtmv_p0/test_paimon_mtmv.groovy | 135 ++++++++++----- 8 files changed, 718 insertions(+), 50 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonPartition.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonPartitionInfo.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtil.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/mtmv/PaimonUtilTest.java 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 c9eaf1b7df32ef1..5645c4e89e726c5 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 @@ -18,10 +18,22 @@ package org.apache.doris.datasource.paimon; 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.catalog.ScalarType; import org.apache.doris.catalog.Type; +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.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.MTMVTimestampSnapshot; +import org.apache.doris.mtmv.MTMVVersionSnapshot; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ExternalAnalysisTask; @@ -30,25 +42,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.commons.collections.CollectionUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.data.InternalRow; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; import org.apache.paimon.table.source.Split; +import org.apache.paimon.table.system.PartitionsTable; +import org.apache.paimon.table.system.SnapshotsTable; import org.apache.paimon.types.ArrayType; 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 java.io.IOException; import java.util.ArrayList; 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 { +public class PaimonExternalTable extends ExternalTable implements MTMVRelatedTableIf, MTMVBaseTableIf { private static final Logger LOG = LogManager.getLogger(PaimonExternalTable.class); @@ -73,18 +95,95 @@ public Table getPaimonTable() { return schemaCacheValue.map(value -> ((PaimonSchemaCacheValue) value).getPaimonTable()).orElse(null); } + private PaimonPartitionInfo getPartitionInfoFromCache() { + makeSureInitialized(); + Optional schemaCacheValue = getSchemaCacheValue(); + if (!schemaCacheValue.isPresent()) { + return new PaimonPartitionInfo(); + } + return ((PaimonSchemaCacheValue) schemaCacheValue.get()).getPartitionInfo(); + } + + private List getPartitionColumnsFromCache() { + makeSureInitialized(); + Optional schemaCacheValue = getSchemaCacheValue(); + if (!schemaCacheValue.isPresent()) { + return Lists.newArrayList(); + } + return ((PaimonSchemaCacheValue) schemaCacheValue.get()).getPartitionColumns(); + } + + public long getLatestSnapshotIdFromCache() throws AnalysisException { + makeSureInitialized(); + Optional schemaCacheValue = getSchemaCacheValue(); + if (!schemaCacheValue.isPresent()) { + throw new AnalysisException("not present"); + } + return ((PaimonSchemaCacheValue) schemaCacheValue.get()).getSnapshootId(); + } + @Override public Optional initSchema() { Table paimonTable = ((PaimonExternalCatalog) catalog).getPaimonTable(dbName, name); TableSchema schema = ((FileStoreTable) paimonTable).schema(); List columns = schema.fields(); List tmpSchema = Lists.newArrayListWithCapacity(columns.size()); + Set partitionColumnNames = Sets.newHashSet(paimonTable.partitionKeys()); + List partitionColumns = Lists.newArrayList(); for (DataField field : columns) { - tmpSchema.add(new Column(field.name().toLowerCase(), + Column column = new Column(field.name().toLowerCase(), paimonTypeToDorisType(field.type()), true, null, true, field.description(), true, - field.id())); + field.id()); + tmpSchema.add(column); + if (partitionColumnNames.contains(field.name())) { + partitionColumns.add(column); + } + } + try { + // after 0.9.0 paimon will support table.getLatestSnapshotId() + long latestSnapshotId = loadLatestSnapshotId(); + PaimonPartitionInfo partitionInfo = loadPartitionInfo(partitionColumns); + return Optional.of(new PaimonSchemaCacheValue(tmpSchema, partitionColumns, paimonTable, latestSnapshotId, + partitionInfo)); + } catch (IOException | AnalysisException e) { + LOG.warn(e); + return Optional.empty(); + } + } + + private long loadLatestSnapshotId() throws IOException { + Table table = ((PaimonExternalCatalog) catalog).getPaimonTable(dbName, + name + Catalog.SYSTEM_TABLE_SPLITTER + SnapshotsTable.SNAPSHOTS); + // snapshotId + List rows = PaimonUtil.read(table, new int[][] {{0}}); + long latestSnapshotId = 0L; + for (InternalRow row : rows) { + long snapshotId = row.getLong(0); + if (snapshotId > latestSnapshotId) { + latestSnapshotId = snapshotId; + } + } + return latestSnapshotId; + } + + private PaimonPartitionInfo loadPartitionInfo(List partitionColumns) throws IOException, AnalysisException { + if (CollectionUtils.isEmpty(partitionColumns)) { + return new PaimonPartitionInfo(); + } + List paimonPartitions = loadPartitions(); + return PaimonUtil.generatePartitionInfo(partitionColumns, paimonPartitions); + } + + private List loadPartitions() + throws IOException { + Table table = ((PaimonExternalCatalog) catalog).getPaimonTable(dbName, + name + Catalog.SYSTEM_TABLE_SPLITTER + PartitionsTable.PARTITIONS); + List rows = PaimonUtil.read(table, null); + List res = Lists.newArrayListWithCapacity(rows.size()); + for (InternalRow row : rows) { + res.add(PaimonUtil.rowToPartition(row)); } - return Optional.of(new PaimonSchemaCacheValue(tmpSchema, paimonTable)); + return res; } private Type paimonPrimitiveTypeToDorisType(org.apache.paimon.types.DataType dataType) { @@ -205,4 +304,56 @@ public long fetchRowCount() { } return rowCount > 0 ? rowCount : UNKNOWN_ROW_COUNT; } + + @Override + public void beforeMTMVRefresh(MTMV mtmv) throws DdlException { + Env.getCurrentEnv().getRefreshManager() + .refreshTable(getCatalog().getName(), getDbName(), getName(), true); + } + + @Override + public Map getAndCopyPartitionItems() { + return Maps.newHashMap(getPartitionInfoFromCache().getNameToPartitionItem()); + } + + @Override + public PartitionType getPartitionType() { + return getPartitionColumnsFromCache().size() > 0 ? PartitionType.LIST : PartitionType.UNPARTITIONED; + } + + @Override + public Set getPartitionColumnNames() { + return getPartitionColumnsFromCache().stream() + .map(c -> c.getName().toLowerCase()).collect(Collectors.toSet()); + } + + @Override + public List getPartitionColumns() { + return getPartitionColumnsFromCache(); + } + + @Override + public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context) + throws AnalysisException { + PaimonPartition paimonPartition = getPartitionInfoFromCache().getNameToPartition().get(partitionName); + if (paimonPartition == null) { + throw new AnalysisException("can not find partition: " + partitionName); + } + return new MTMVTimestampSnapshot(paimonPartition.getLastUpdateTime()); + } + + @Override + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context) throws AnalysisException { + return new MTMVVersionSnapshot(getLatestSnapshotIdFromCache()); + } + + @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; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonPartition.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonPartition.java new file mode 100644 index 000000000000000..545448199b33757 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonPartition.java @@ -0,0 +1,61 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.paimon; + +// https://paimon.apache.org/docs/0.9/maintenance/system-tables/#partitions-table +public class PaimonPartition { + // Partition values, for example: [1, dd] + private final String partitionValues; + // The amount of data in the partition + private final long recordCount; + // Partition file size + private final long fileSizeInBytes; + // Number of partition files + private final long fileCount; + // Last update time of partition + private final long lastUpdateTime; + + public PaimonPartition(String partitionValues, long recordCount, long fileSizeInBytes, long fileCount, + long lastUpdateTime) { + this.partitionValues = partitionValues; + this.recordCount = recordCount; + this.fileSizeInBytes = fileSizeInBytes; + this.fileCount = fileCount; + this.lastUpdateTime = lastUpdateTime; + } + + public String getPartitionValues() { + return partitionValues; + } + + public long getRecordCount() { + return recordCount; + } + + public long getFileSizeInBytes() { + return fileSizeInBytes; + } + + public long getFileCount() { + return fileCount; + } + + public long getLastUpdateTime() { + return lastUpdateTime; + } +} 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 new file mode 100644 index 000000000000000..8f54f0834e481b6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonPartitionInfo.java @@ -0,0 +1,48 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.paimon; + +import org.apache.doris.catalog.PartitionItem; + +import com.google.common.collect.Maps; + +import java.util.Map; + +public class PaimonPartitionInfo { + private Map nameToPartitionItem; + private Map nameToPartition; + + public PaimonPartitionInfo() { + this.nameToPartitionItem = Maps.newHashMap(); + this.nameToPartition = Maps.newHashMap(); + } + + public PaimonPartitionInfo(Map nameToPartitionItem, + Map nameToPartition) { + this.nameToPartitionItem = nameToPartitionItem; + this.nameToPartition = nameToPartition; + } + + public Map getNameToPartitionItem() { + return nameToPartitionItem; + } + + 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 aaaefe7f32db2bf..20d27b2425df249 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 @@ -27,13 +27,34 @@ public class PaimonSchemaCacheValue extends SchemaCacheValue { private Table paimonTable; + private List partitionColumns; + private PaimonPartitionInfo partitionInfo; - public PaimonSchemaCacheValue(List schema, Table paimonTable) { + private long snapshootId; + + public PaimonSchemaCacheValue(List schema, List partitionColumns, Table paimonTable, + long snapshootId, + PaimonPartitionInfo partitionInfo) { super(schema); + this.partitionColumns = partitionColumns; this.paimonTable = paimonTable; + this.snapshootId = snapshootId; + this.partitionInfo = partitionInfo; } public Table getPaimonTable() { return paimonTable; } + + public List getPartitionColumns() { + return partitionColumns; + } + + public PaimonPartitionInfo getPartitionInfo() { + return partitionInfo; + } + + public long getSnapshootId() { + return snapshootId; + } } 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 new file mode 100644 index 000000000000000..8b7017cac294864 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonUtil.java @@ -0,0 +1,155 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.paimon; + +import org.apache.doris.analysis.PartitionValue; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.ListPartitionItem; +import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.catalog.PartitionKey; +import org.apache.doris.catalog.Type; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.datasource.hive.HiveUtil; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.commons.collections.CollectionUtils; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.serializer.InternalRowSerializer; +import org.apache.paimon.options.ConfigOption; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.utils.Pair; +import org.apache.paimon.utils.Projection; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import javax.annotation.Nullable; + +public class PaimonUtil { + public static List read( + Table table, @Nullable int[][] projection, Pair, String>... dynamicOptions) + throws IOException { + Map options = new HashMap<>(); + for (Pair, String> pair : dynamicOptions) { + options.put(pair.getKey().key(), pair.getValue()); + } + table = table.copy(options); + ReadBuilder readBuilder = table.newReadBuilder(); + if (projection != null) { + readBuilder.withProjection(projection); + } + RecordReader reader = + readBuilder.newRead().createReader(readBuilder.newScan().plan()); + InternalRowSerializer serializer = + new InternalRowSerializer( + projection == null + ? table.rowType() + : Projection.of(projection).project(table.rowType())); + List rows = new ArrayList<>(); + reader.forEachRemaining(row -> rows.add(serializer.copy(row))); + return rows; + } + + + /* + 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) throws AnalysisException { + Map nameToPartitionItem = 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); + nameToPartitionItem.put(partitionName, toListPartitionItem(partitionName, partitionColumns)); + } + 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() + .map(Column::getType) + .collect(Collectors.toList()); + // Partition name will be in format: nation=cn/city=beijing + // parse it to get values "cn" and "beijing" + List partitionValues = HiveUtil.toPartitionValues(partitionName); + Preconditions.checkState(partitionValues.size() == types.size(), partitionName + " vs. " + types); + List values = Lists.newArrayListWithExpectedSize(types.size()); + for (String partitionValue : partitionValues) { + // null will in partition 'null' + // "null" will in partition 'null' + // NULL will in partition 'null' + // "NULL" will in partition 'NULL' + // values.add(new PartitionValue(partitionValue, "null".equals(partitionValue))); + values.add(new PartitionValue(partitionValue, false)); + } + PartitionKey key = PartitionKey.createListPartitionKeyWithTypes(values, types, true); + ListPartitionItem listPartitionItem = new ListPartitionItem(Lists.newArrayList(key)); + return listPartitionItem; + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/PaimonUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/PaimonUtilTest.java new file mode 100644 index 000000000000000..789af7bf8357acb --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/PaimonUtilTest.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.mtmv; + +import org.apache.doris.analysis.LiteralExpr; +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.common.AnalysisException; +import org.apache.doris.datasource.paimon.PaimonPartition; +import org.apache.doris.datasource.paimon.PaimonPartitionInfo; +import org.apache.doris.datasource.paimon.PaimonUtil; + +import com.google.common.collect.Lists; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.Timestamp; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class PaimonUtilTest { + + @Test + public void testGeneratePartitionInfo() throws AnalysisException { + Column k1 = new Column("k1", PrimitiveType.INT); + Column k2 = new Column("k2", PrimitiveType.VARCHAR); + List partitionColumns = Lists.newArrayList(k1, k2); + PaimonPartition p1 = new PaimonPartition("[1,aa]", 2, 3, 4, 5); + List paimonPartitions = Lists.newArrayList(p1); + PaimonPartitionInfo partitionInfo = PaimonUtil.generatePartitionInfo(partitionColumns, paimonPartitions); + String expectPartitionName = "k1=1/k2=aa"; + Assert.assertTrue(partitionInfo.getNameToPartitionItem().containsKey(expectPartitionName)); + PartitionItem partitionItem = partitionInfo.getNameToPartitionItem().get(expectPartitionName); + List keys = partitionItem.getItems(); + Assert.assertEquals(1, keys.size()); + PartitionKey partitionKey = keys.get(0); + List exprs = partitionKey.getKeys(); + Assert.assertEquals(2, exprs.size()); + Assert.assertEquals(1, exprs.get(0).getLongValue()); + Assert.assertEquals("aa", exprs.get(1).getStringValue()); + } + + @Test + public void testRowToPartition() { + GenericRow row = GenericRow.of(BinaryString.fromString("[1,b]"), 2L, 3L, 4L, Timestamp.fromEpochMillis(5L)); + PaimonPartition paimonPartition = PaimonUtil.rowToPartition(row); + Assert.assertEquals("[1,b]", paimonPartition.getPartitionValues()); + Assert.assertEquals(2L, paimonPartition.getRecordCount()); + Assert.assertEquals(3L, paimonPartition.getFileSizeInBytes()); + Assert.assertEquals(4L, paimonPartition.getFileCount()); + Assert.assertEquals(5L, paimonPartition.getLastUpdateTime()); + } +} diff --git a/regression-test/data/mtmv_p0/test_paimon_mtmv.out b/regression-test/data/mtmv_p0/test_paimon_mtmv.out index c654cb01214f57b..c28b7cb7baca229 100644 --- a/regression-test/data/mtmv_p0/test_paimon_mtmv.out +++ b/regression-test/data/mtmv_p0/test_paimon_mtmv.out @@ -1,9 +1,113 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !catalog -- -1 2 3 4 5 6 7 8 9.1 10.1 11.10 2020-02-02 13str 14varchar a true aaaa 2023-08-13T09:32:38.530 -10 20 30 40 50 60 70 80 90.1 100.1 110.10 2020-03-02 130str 140varchar b false bbbb 2023-08-14T08:32:52.821 +-- !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 --- !mtmv -- -1 2 3 4 5 6 7 8 9.1 10.1 11.10 2020-02-02 13str 14varchar a true aaaa 2023-08-13T09:32:38.530 -10 20 30 40 50 60 70 80 90.1 100.1 110.10 2020-03-02 130str 140varchar b false bbbb 2023-08-14T08:32:52.821 +-- !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 index e84eb497b2c7b15..f2989edbf6dfd60 100644 --- a/regression-test/suites/mtmv_p0/test_paimon_mtmv.groovy +++ b/regression-test/suites/mtmv_p0/test_paimon_mtmv.groovy @@ -15,48 +15,105 @@ // specific language governing permissions and limitations // under the License. -suite("test_paimon_mtmv", "p0,external,paimon,external_docker,external_docker_hive") { +suite("test_paimon_mtmv", "p0,external,mtmv,external_docker,external_docker_doris") { String enabled = context.config.otherConfigs.get("enablePaimonTest") - logger.info("enabled: " + enabled) + 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") - logger.info("externalEnvIp: " + externalEnvIp) - String hdfs_port = context.config.otherConfigs.get("hive2HdfsPort") - logger.info("hdfs_port: " + hdfs_port) - if (enabled != null && enabled.equalsIgnoreCase("true")) { - String catalog_name = "paimon_mtmv_catalog"; - String mvName = "test_paimon_mtmv" - String dbName = "regression_test_mtmv_p0" - String paimonDb = "db1" - String paimonTable = "all_table" - sql """drop catalog if exists ${catalog_name} """ - - sql """create catalog if not exists ${catalog_name} properties ( - "type" = "paimon", - "paimon.catalog.type"="filesystem", - "warehouse" = "hdfs://${externalEnvIp}:${hdfs_port}/user/doris/paimon1" + + 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_catalog """select * from ${catalog_name}.${paimonDb}.${paimonTable}""" - sql """drop materialized view if exists ${mvName};""" - - sql """ - CREATE MATERIALIZED VIEW ${mvName} - BUILD DEFERRED REFRESH AUTO ON MANUAL - DISTRIBUTED BY RANDOM BUCKETS 2 - PROPERTIES ('replication_num' = '1') - AS - SELECT * FROM ${catalog_name}.${paimonDb}.${paimonTable}; - """ - - sql """ - REFRESH MATERIALIZED VIEW ${mvName} complete - """ - def jobName = getJobName(dbName, mvName); - waitingMTMVTaskFinished(jobName) - order_qt_mtmv "SELECT * FROM ${mvName}" - - sql """drop materialized view if exists ${mvName};""" - sql """ drop catalog if exists ${catalog_name} """ - } + 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 dd4583f1ef14043bd718766dd37234c20e2ee3f6 Mon Sep 17 00:00:00 2001 From: James Date: Thu, 21 Nov 2024 10:51:05 +0800 Subject: [PATCH 003/110] [improvement](statistics)Change auto analyze max width to 300 and health threshold to 90. (#42104) ### What problem does this PR solve? Change auto analyze max width to 300 and health threshold to 90. This could auto analyze wider tables by default and collect table with data changes more frequently. Issue Number: close #xxx Related PR: #xxx Problem Summary: ### Release note None --- .../org/apache/doris/qe/GlobalVariable.java | 1 + .../org/apache/doris/qe/SessionVariable.java | 4 +-- .../java/org/apache/doris/qe/VariableMgr.java | 16 +++++++++++ .../doris/statistics/StatisticConstants.java | 4 +-- .../statistics/util/StatisticsUtilTest.java | 27 +++++++++++++++---- 5 files changed, 43 insertions(+), 9 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/GlobalVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/GlobalVariable.java index 45a6acba6dc38f5..e3639596ed36bcc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/GlobalVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/GlobalVariable.java @@ -35,6 +35,7 @@ public final class GlobalVariable { public static final int VARIABLE_VERSION_0 = 0; public static final int VARIABLE_VERSION_100 = 100; + public static final int VARIABLE_VERSION_101 = 101; public static final int VARIABLE_VERSION_200 = 200; public static final int VARIABLE_VERSION_300 = 300; public static final int CURRENT_VARIABLE_VERSION = VARIABLE_VERSION_300; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index eb0a669e05be5e6..bd724af54c49e87 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -1921,7 +1921,7 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { "Maximum table width to enable auto analyze, " + "table with more columns than this value will not be auto analyzed."}, flag = VariableMgr.GLOBAL) - public int autoAnalyzeTableWidthThreshold = 100; + public int autoAnalyzeTableWidthThreshold = 300; @VariableMgr.VarAttr(name = AUTO_ANALYZE_START_TIME, needForward = true, checker = "checkAnalyzeTimeFormat", description = {"该参数定义自动ANALYZE例程的开始时间", @@ -1986,7 +1986,7 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { + "exceeds (100 - table_stats_health_threshold)% since the last " + "statistics collection operation, the statistics for this table are" + "considered outdated."}) - public int tableStatsHealthThreshold = 60; + public int tableStatsHealthThreshold = 90; @VariableMgr.VarAttr(name = ENABLE_MATERIALIZED_VIEW_REWRITE, needForward = true, description = {"是否开启基于结构信息的物化视图透明改写", diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/VariableMgr.java b/fe/fe-core/src/main/java/org/apache/doris/qe/VariableMgr.java index b7a4f8bb710074a..5b632c1632e968e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/VariableMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/VariableMgr.java @@ -34,6 +34,8 @@ import org.apache.doris.common.util.SerializationUtils; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.persist.GlobalVarPersistInfo; +import org.apache.doris.statistics.StatisticConstants; +import org.apache.doris.statistics.util.StatisticsUtil; import com.google.common.base.Preconditions; import com.google.common.base.Strings; @@ -984,6 +986,20 @@ public static void forceUpdateVariables() { SessionVariable.ENABLE_PIPELINE_X_ENGINE, String.valueOf(true)); } + if (currentVariableVersion < GlobalVariable.VARIABLE_VERSION_101) { + if (StatisticsUtil.getAutoAnalyzeTableWidthThreshold() + < StatisticConstants.AUTO_ANALYZE_TABLE_WIDTH_THRESHOLD) { + VariableMgr.refreshDefaultSessionVariables("update variable version", + SessionVariable.AUTO_ANALYZE_TABLE_WIDTH_THRESHOLD, + String.valueOf(StatisticConstants.AUTO_ANALYZE_TABLE_WIDTH_THRESHOLD)); + } + if (StatisticsUtil.getTableStatsHealthThreshold() + < StatisticConstants.TABLE_STATS_HEALTH_THRESHOLD) { + VariableMgr.refreshDefaultSessionVariables("update variable version", + SessionVariable.TABLE_STATS_HEALTH_THRESHOLD, + String.valueOf(StatisticConstants.TABLE_STATS_HEALTH_THRESHOLD)); + } + } if (currentVariableVersion < GlobalVariable.VARIABLE_VERSION_200) { // update from 3.0.2 or below to 3.0.3 or higher VariableMgr.refreshDefaultSessionVariables("update variable version", diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java index c2479147ec7b20f..f6d49ea079bf188 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticConstants.java @@ -93,13 +93,13 @@ public class StatisticConstants { public static final long EXTERNAL_TABLE_AUTO_ANALYZE_INTERVAL_IN_MILLIS = TimeUnit.HOURS.toMillis(24); - public static final int TABLE_STATS_HEALTH_THRESHOLD = 60; + public static final int TABLE_STATS_HEALTH_THRESHOLD = 90; public static final int ANALYZE_TIMEOUT_IN_SEC = 43200; public static final int TASK_QUEUE_CAP = 1; - public static final int AUTO_ANALYZE_TABLE_WIDTH_THRESHOLD = 100; + public static final int AUTO_ANALYZE_TABLE_WIDTH_THRESHOLD = 300; public static final int MSG_LEN_UPPER_BOUND = 1024; diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/util/StatisticsUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/util/StatisticsUtilTest.java index 32521882939d348..fbac718e421c296 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/util/StatisticsUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/util/StatisticsUtilTest.java @@ -344,11 +344,11 @@ public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { tableMeta.partitionChanged.set(false); Assertions.assertTrue(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); - // Test update rows changed more than threshold. + // Test row count changed more than threshold. new MockUp() { @Mock public long getRowCount() { - return 120; + return 111; } }; new MockUp() { @@ -358,12 +358,29 @@ public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { } }; tableMeta.partitionChanged.set(false); - tableMeta.updatedRows.set(200); + tableMeta.updatedRows.set(80); Assertions.assertTrue(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); - // Test update rows changed less than threshold + // Test update rows changed more than threshold + new MockUp() { + @Mock + public long getRowCount() { + return 101; + } + }; + tableMeta.partitionChanged.set(false); + tableMeta.updatedRows.set(91); + Assertions.assertTrue(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); + + // Test row count and update rows changed less than threshold + new MockUp() { + @Mock + public long getRowCount() { + return 100; + } + }; tableMeta.partitionChanged.set(false); - tableMeta.updatedRows.set(100); + tableMeta.updatedRows.set(85); Assertions.assertFalse(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); } From 8c3f3ffc2d5551478679ae36134d0aa6fb4ecdc4 Mon Sep 17 00:00:00 2001 From: cat-with-cat Date: Thu, 21 Nov 2024 11:16:20 +0800 Subject: [PATCH 004/110] [improvement](log)Support exception report in our benchmark utils (#40834) (#42445) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Proposed changes Issue Number: close #40834 ``` ➜ doris_ bash /home/cat/doris_/doris/tools/clickbench-tools/run-clickbench-queries.sh query1,ERROR 1105 (HY000) at line 1: errCode = 2, detailMessage = There is no scanNode Backend available.[10004: not alive] ,ERROR 1105 (HY000) at line 1: errCode = 2, detailMessage = There is no scanNode Backend available.[10004: not alive] ,ERROR 1105 (HY000) at line 1: errCode = 2, detailMessage = There is no scanNode Backend available.[10004: not alive] ``` ``` ➜ doris_ bash /home/cat/doris_/doris/tools/ssb-tools/bin/run-ssb-flat-queries.sh ============================================ q1.1 317 112 107 107 q1.2 91 52 52 52 q1.3 198 101 136 101 q2.1 3197 1751 1642 1642 q2.2 Error: Failed to execute query q2.2 (cold run). Output: ERROR 1105 (HY000) at line 17: RpcException, msg: send fragments failed. io.grpc.StatusRuntimeException: UNAVAILABLE: io exception, host: 192.168.41.145 q2.3 Error: Failed to execute query q2.3 (cold run). Output: ERROR 1105 (HY000) at line 17: RpcException, msg: send fragments failed. io.grpc.StatusRuntimeException: UNAVAILABLE: io exception, host: 192.168.41.145 ``` ``` ➜ doris_ bash /home/cat/doris_/doris/tools/ssb-tools/bin/run-ssb-queries.sh ============================================ q1.1 881 210 231 210 q1.2 1075 153 176 153 q1.3 131 124 143 124 q2.1 5429 1551 1638 1551 q2.2 1785 1618 1615 1615 q2.3 1234 1189 1418 1189 q3.1 Error: Failed to execute query q3.1 (cold run). Output: ERROR 1105 (HY000) at line 17: RpcException, msg: send fragments failed. io.grpc.StatusRuntimeException: UNAVAILABLE: io exception, host: 192.168.41.145 q3.2 Error: Failed to execute query q3.2 (cold run). Output: ERROR 1105 (HY000) at line 17: RpcException, msg: send fragments failed. io.grpc.StatusRuntimeException: UNAVAILABLE: io exception, host: 192.168.41.145 ``` ``` ➜ doris_ bash /home/cat/doris_/doris/tools/clickbench-tools/load-clickbench-data.sh FE_HOST: 127.0.0.1 FE_HTTP_PORT: 8030 USER: root PASSWORD: DB: clickbench start... Curl failed with the following output: Error: Failed to execute curl to fetch BE's configuration. ``` ``` ➜ doris_ bash /home/cat/doris_/doris/tools/clickbench-tools/run-clickbench-queries.sh mysql Ver 8.0.39-0ubuntu0.22.04.1 for Linux on x86_64 ((Ubuntu)) This is perl 5, version 34, subversion 0 (v5.34.0) built for x86_64-linux-gnu-thread-multi (with 60 registered patches, see perl -V for more detail) Copyright 1987-2021, Larry Wall Perl may be copied only under the terms of either the Artistic License or the GNU General Public License, which may be found in the Perl 5 source kit. Complete documentation for Perl, including FAQ lists, should be found on this system using "man perl" or "perldoc perl". If you have access to the Internet, point your browser at http://www.perl.org/, the Perl Home Page. FE_HOST: 127.0.0.1 FE_QUERY_PORT: 9030 USER: root PASSWORD: DB: clickbench ERROR 2003 (HY000): Can't connect to MySQL server on '127.0.0.1:9030' (111) ``` --- tools/clickbench-tools/load-clickbench-data.sh | 7 ++++++- tools/ssb-tools/bin/run-ssb-flat-queries.sh | 18 +++++++++++++++--- tools/ssb-tools/bin/run-ssb-queries.sh | 18 +++++++++++++++--- tools/tpcds-tools/bin/run-tpcds-queries.sh | 18 +++++++++++++++--- tools/tpch-tools/bin/run-tpch-queries.sh | 18 +++++++++++++++--- 5 files changed, 66 insertions(+), 13 deletions(-) diff --git a/tools/clickbench-tools/load-clickbench-data.sh b/tools/clickbench-tools/load-clickbench-data.sh index 89cc3d22df4ae69..208bdaff52d2fee 100755 --- a/tools/clickbench-tools/load-clickbench-data.sh +++ b/tools/clickbench-tools/load-clickbench-data.sh @@ -110,7 +110,12 @@ function check_doris_conf() { echo "advise: revise your Doris FE's conf to set 'stream_load_default_timeout_second=3600' or above" fi - cv=$(curl "${BE_HOST}:${BE_WEBSERVER_PORT}/varz" 2>/dev/null | grep 'streaming_load_max_mb' | awk -F'=' '{print $2}') + if ! output=$(curl -s "${BE_HOST}:${BE_WEBSERVER_PORT}/varz" 2>&1); then + printf "Curl failed with the following output:\n%s\n" "${output}" >&2 + printf "Error: Failed to execute curl to fetch BE's configuration.\n" >&2 + exit 1 + fi + cv=$(grep 'streaming_load_max_mb' <<< "${output}" | awk -F'=' '{print $2}') if (($cv < 16000)); then echo -e "advise: revise your Doris BE's conf to set 'streaming_load_max_mb=16000' or above and 'flush_thread_num_per_store=5' to speed up load." fi diff --git a/tools/ssb-tools/bin/run-ssb-flat-queries.sh b/tools/ssb-tools/bin/run-ssb-flat-queries.sh index ab57bccc728f42e..bbd2c2146dd2964 100755 --- a/tools/ssb-tools/bin/run-ssb-flat-queries.sh +++ b/tools/ssb-tools/bin/run-ssb-flat-queries.sh @@ -120,19 +120,31 @@ for i in '1.1' '1.2' '1.3' '2.1' '2.2' '2.3' '3.1' '3.2' '3.3' '3.4' '4.1' '4.2' hot2=0 echo -ne "q${i}\t" | tee -a result.csv start=$(date +%s%3N) - mysql -h"${FE_HOST}" -u "${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments <"${QUERIES_DIR}"/q"${i}".sql >"${RESULT_DIR}"/result"${i}".out 2>"${RESULT_DIR}"/result"${i}".log + if ! output=$(mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments \ + <"${QUERIES_DIR}/q${i}.sql" 2>&1); then + printf "Error: Failed to execute query q%s (cold run). Output:\n%s\n" "${i}" "${output}" >&2 + continue + fi end=$(date +%s%3N) cold=$((end - start)) echo -ne "${cold}\t" | tee -a result.csv start=$(date +%s%3N) - mysql -h"${FE_HOST}" -u "${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments <"${QUERIES_DIR}"/q"${i}".sql >"${RESULT_DIR}"/result"${i}".out 2>"${RESULT_DIR}"/result"${i}".log + if ! output=$(mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments \ + <"${QUERIES_DIR}/q${i}.sql" 2>&1); then + printf "Error: Failed to execute query q%s (hot run 1). Output:\n%s\n" "${i}" "${output}" >&2 + continue + fi end=$(date +%s%3N) hot1=$((end - start)) echo -ne "${hot1}\t" | tee -a result.csv start=$(date +%s%3N) - mysql -h"${FE_HOST}" -u "${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments <"${QUERIES_DIR}"/q"${i}".sql >"${RESULT_DIR}"/result"${i}".out 2>"${RESULT_DIR}"/result"${i}".log + if ! output=$(mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments \ + <"${QUERIES_DIR}/q${i}.sql" 2>&1); then + printf "Error: Failed to execute query q%s (hot run 2). Output:\n%s\n" "${i}" "${output}" >&2 + continue + fi end=$(date +%s%3N) hot2=$((end - start)) echo -ne "${hot2}\t" | tee -a result.csv diff --git a/tools/ssb-tools/bin/run-ssb-queries.sh b/tools/ssb-tools/bin/run-ssb-queries.sh index 58cfb5ef7be3cef..ffde7e3962eaf49 100755 --- a/tools/ssb-tools/bin/run-ssb-queries.sh +++ b/tools/ssb-tools/bin/run-ssb-queries.sh @@ -120,19 +120,31 @@ for i in '1.1' '1.2' '1.3' '2.1' '2.2' '2.3' '3.1' '3.2' '3.3' '3.4' '4.1' '4.2' hot2=0 echo -ne "q${i}\t" | tee -a result.csv start=$(date +%s%3N) - mysql -h"${FE_HOST}" -u "${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments <"${QUERIES_DIR}"/q"${i}".sql >"${RESULT_DIR}"/result"${i}".out 2>"${RESULT_DIR}"/result"${i}".log + if ! output=$(mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments \ + <"${QUERIES_DIR}/q${i}.sql" 2>&1); then + printf "Error: Failed to execute query q%s (cold run). Output:\n%s\n" "${i}" "${output}" >&2 + continue + fi end=$(date +%s%3N) cold=$((end - start)) echo -ne "${cold}\t" | tee -a result.csv start=$(date +%s%3N) - mysql -h"${FE_HOST}" -u "${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments <"${QUERIES_DIR}"/q"${i}".sql >"${RESULT_DIR}"/result"${i}".out 2>"${RESULT_DIR}"/result"${i}".log + if ! output=$(mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments \ + <"${QUERIES_DIR}/q${i}.sql" 2>&1); then + printf "Error: Failed to execute query q%s (hot run 1). Output:\n%s\n" "${i}" "${output}" >&2 + continue + fi end=$(date +%s%3N) hot1=$((end - start)) echo -ne "${hot1}\t" | tee -a result.csv start=$(date +%s%3N) - mysql -h"${FE_HOST}" -u "${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments <"${QUERIES_DIR}"/q"${i}".sql >"${RESULT_DIR}"/result"${i}".out 2>"${RESULT_DIR}"/result"${i}".log + if ! output=$(mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments \ + <"${QUERIES_DIR}/q${i}.sql" 2>&1); then + printf "Error: Failed to execute query q%s (hot run 2). Output:\n%s\n" "${i}" "${output}" >&2 + continue + fi end=$(date +%s%3N) hot2=$((end - start)) echo -ne "${hot2}\t" | tee -a result.csv diff --git a/tools/tpcds-tools/bin/run-tpcds-queries.sh b/tools/tpcds-tools/bin/run-tpcds-queries.sh index ea2b3dd92a2f106..1af50eea3e98f62 100755 --- a/tools/tpcds-tools/bin/run-tpcds-queries.sh +++ b/tools/tpcds-tools/bin/run-tpcds-queries.sh @@ -144,19 +144,31 @@ for i in ${query_array[@]}; do hot2=0 echo -ne "query${i}\t" | tee -a result.csv start=$(date +%s%3N) - mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments <"${TPCDS_QUERIES_DIR}"/query"${i}".sql >"${RESULT_DIR}"/result"${i}".out 2>"${RESULT_DIR}"/result"${i}".log + if ! output=$(mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments \ + <"${TPCDS_QUERIES_DIR}/query${i}.sql" 2>&1); then + printf "Error: Failed to execute query q%s (cold run). Output:\n%s\n" "${i}" "${output}" >&2 + continue + fi end=$(date +%s%3N) cold=$((end - start)) echo -ne "${cold}\t" | tee -a result.csv start=$(date +%s%3N) - mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments <"${TPCDS_QUERIES_DIR}"/query"${i}".sql >"${RESULT_DIR}"/result"${i}".out 2>"${RESULT_DIR}"/result"${i}".log + if ! output=$(mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments \ + <"${TPCDS_QUERIES_DIR}/query${i}.sql" 2>&1); then + printf "Error: Failed to execute query q%s (hot run 1). Output:\n%s\n" "${i}" "${output}" >&2 + continue + fi end=$(date +%s%3N) hot1=$((end - start)) echo -ne "${hot1}\t" | tee -a result.csv start=$(date +%s%3N) - mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments <"${TPCDS_QUERIES_DIR}"/query"${i}".sql >"${RESULT_DIR}"/result"${i}".out 2>"${RESULT_DIR}"/result"${i}".log + if ! output=$(mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments \ + <"${TPCDS_QUERIES_DIR}/query${i}.sql" 2>&1); then + printf "Error: Failed to execute query q%s (hot run 2). Output:\n%s\n" "${i}" "${output}" >&2 + continue + fi end=$(date +%s%3N) hot2=$((end - start)) echo -ne "${hot2}\t" | tee -a result.csv diff --git a/tools/tpch-tools/bin/run-tpch-queries.sh b/tools/tpch-tools/bin/run-tpch-queries.sh index 56abe1799ba309e..43e5a16bb53cdd2 100755 --- a/tools/tpch-tools/bin/run-tpch-queries.sh +++ b/tools/tpch-tools/bin/run-tpch-queries.sh @@ -141,19 +141,31 @@ for i in ${query_array[@]}; do hot2=0 echo -ne "q${i}\t" | tee -a result.csv start=$(date +%s%3N) - mysql -h"${FE_HOST}" -u "${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments <"${TPCH_QUERIES_DIR}"/q"${i}".sql >"${RESULT_DIR}"/result"${i}".out 2>"${RESULT_DIR}"/result"${i}".log + if ! output=$(mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments \ + <"${TPCH_QUERIES_DIR}/q${i}.sql" 2>&1); then + printf "Error: Failed to execute query q%s (cold run). Output:\n%s\n" "${i}" "${output}" >&2 + continue + fi end=$(date +%s%3N) cold=$((end - start)) echo -ne "${cold}\t" | tee -a result.csv start=$(date +%s%3N) - mysql -h"${FE_HOST}" -u "${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments <"${TPCH_QUERIES_DIR}"/q"${i}".sql >"${RESULT_DIR}"/result"${i}".out 2>"${RESULT_DIR}"/result"${i}".log + if ! output=$(mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments \ + <"${TPCH_QUERIES_DIR}/q${i}.sql" 2>&1); then + printf "Error: Failed to execute query q%s (hot run 1). Output:\n%s\n" "${i}" "${output}" >&2 + continue + fi end=$(date +%s%3N) hot1=$((end - start)) echo -ne "${hot1}\t" | tee -a result.csv start=$(date +%s%3N) - mysql -h"${FE_HOST}" -u "${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments <"${TPCH_QUERIES_DIR}"/q"${i}".sql >"${RESULT_DIR}"/result"${i}".out 2>"${RESULT_DIR}"/result"${i}".log + if ! output=$(mysql -h"${FE_HOST}" -u"${USER}" -P"${FE_QUERY_PORT}" -D"${DB}" --comments \ + <"${TPCH_QUERIES_DIR}/q${i}.sql" 2>&1); then + printf "Error: Failed to execute query q%s (hot run 2). Output:\n%s\n" "${i}" "${output}" >&2 + continue + fi end=$(date +%s%3N) hot2=$((end - start)) echo -ne "${hot2}\t" | tee -a result.csv From 012620c7d4faf75ed4b1c6833d1227867dea4c95 Mon Sep 17 00:00:00 2001 From: Siyang Tang Date: Thu, 21 Nov 2024 11:24:01 +0800 Subject: [PATCH 005/110] [Fix](ms) Fix ci-uncaught ut coredump due to dynamic rate limit setting (#44362) UT failed with coredump in PR #42413 but was not detected by CI. Fix the problem. --- cloud/script/run_all_tests.sh | 2 ++ cloud/src/meta-service/meta_service_http.cpp | 14 +++++++------- run-cloud-ut.sh | 1 + 3 files changed, 10 insertions(+), 7 deletions(-) diff --git a/cloud/script/run_all_tests.sh b/cloud/script/run_all_tests.sh index eaddd4ff78fad39..330fb00449350bf 100644 --- a/cloud/script/run_all_tests.sh +++ b/cloud/script/run_all_tests.sh @@ -143,11 +143,13 @@ for i in *_test; do patchelf --set-rpath "$(pwd)" "${i}" fi + set -euo pipefail if [[ "${filter}" == "" ]]; then LLVM_PROFILE_FILE="./report/${i}.profraw" "./${i}" --gtest_print_time=true --gtest_output="xml:${i}.xml" else LLVM_PROFILE_FILE="./report/${i}.profraw" "./${i}" --gtest_print_time=true --gtest_output="xml:${i}.xml" --gtest_filter="${filter}" fi + set +euo pipefail unittest_files[${#unittest_files[*]}]="${i}" echo "--------------------------" fi diff --git a/cloud/src/meta-service/meta_service_http.cpp b/cloud/src/meta-service/meta_service_http.cpp index 2f7536e9989a6cf..e919c6f365e3b8e 100644 --- a/cloud/src/meta-service/meta_service_http.cpp +++ b/cloud/src/meta-service/meta_service_http.cpp @@ -411,8 +411,8 @@ static HttpResponse process_adjust_rate_limit(MetaServiceImpl* service, brpc::Co processors[0b101] = std::move(set_instance_qps_limit); processors[0b111] = std::move(set_instance_rpc_qps_limit); - uint8_t level = (0x01 & qps_limit_str.empty()) | ((0x01 & rpc_name.empty()) << 1) | - ((0x01 & instance_id.empty()) << 2); + uint8_t level = (0x01 & !qps_limit_str.empty()) | ((0x01 & !rpc_name.empty()) << 1) | + ((0x01 & !instance_id.empty()) << 2); DCHECK_LT(level, 8); @@ -422,10 +422,13 @@ static HttpResponse process_adjust_rate_limit(MetaServiceImpl* service, brpc::Co static HttpResponse process_query_rate_limit(MetaServiceImpl* service, brpc::Controller* cntl) { auto rate_limiter = service->rate_limiter(); rapidjson::Document d; + d.SetObject(); auto get_qps_limit = [&d](std::string_view rpc_name, std::shared_ptr rpc_limiter) { rapidjson::Document node; + node.SetObject(); rapidjson::Document sub; + sub.SetObject(); auto get_qps_token_limit = [&](std::string_view instance_id, std::shared_ptr qps_token) { sub.AddMember(rapidjson::StringRef(instance_id.data(), instance_id.size()), @@ -433,10 +436,7 @@ static HttpResponse process_query_rate_limit(MetaServiceImpl* service, brpc::Con }; rpc_limiter->for_each_qps_token(std::move(get_qps_token_limit)); - auto max_qps_limit = std::to_string(rpc_limiter->max_qps_limit()); - node.AddMember("RPC qps limit", - rapidjson::StringRef(max_qps_limit.data(), max_qps_limit.size()), - d.GetAllocator()); + node.AddMember("RPC qps limit", rpc_limiter->max_qps_limit(), d.GetAllocator()); node.AddMember("instance specific qps limit", sub, d.GetAllocator()); d.AddMember(rapidjson::StringRef(rpc_name.data(), rpc_name.size()), node, d.GetAllocator()); }; @@ -445,7 +445,7 @@ static HttpResponse process_query_rate_limit(MetaServiceImpl* service, brpc::Con rapidjson::StringBuffer sb; rapidjson::PrettyWriter writer(sb); d.Accept(writer); - return http_json_reply(MetaServiceCode::OK, sb.GetString()); + return http_json_reply(MetaServiceCode::OK, "", sb.GetString()); } static HttpResponse process_decode_key(MetaServiceImpl*, brpc::Controller* ctrl) { diff --git a/run-cloud-ut.sh b/run-cloud-ut.sh index 16473b1a2a88733..c74eee7b5f3eb40 100755 --- a/run-cloud-ut.sh +++ b/run-cloud-ut.sh @@ -228,6 +228,7 @@ cd test # FILTER: binary_name:gtest_filter # FILTER: meta_service_test:DetachSchemaKVTest.* # ./run_all_tests.sh --test "\"$(echo "${FILTER}" | awk -F: '{print $1}')\"" --filter "\"$(echo "${FILTER}" | awk -F: '{print $2}')\"" --fdb "\"${FDB}\"" +set -euo pipefail if [[ "_${ENABLE_CLANG_COVERAGE}" == "_ON" ]]; then bash -x ./run_all_tests.sh --coverage --test "$(echo "${FILTER}" | awk -F: '{print $1}')" --filter "$(echo "${FILTER}" | awk -F: '{print $2}')" --fdb "${FDB}" else From 3f7af185b67fadea91891eddabe9af908d00152c Mon Sep 17 00:00:00 2001 From: Pxl Date: Thu, 21 Nov 2024 11:29:16 +0800 Subject: [PATCH 006/110] [Chore](runtime-filter) change some rf dcheck to exception (#44354) ### What problem does this PR solve? change some rf dcheck to exception --- be/src/exprs/runtime_filter.cpp | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index 3eb3d406ad2901d..2ca6493dc31c861 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -362,8 +362,11 @@ class RuntimePredicateWrapper { } Status init_bloom_filter(const size_t build_bf_cardinality) { - DCHECK(_filter_type == RuntimeFilterType::BLOOM_FILTER || - _filter_type == RuntimeFilterType::IN_OR_BLOOM_FILTER); + if (_filter_type != RuntimeFilterType::BLOOM_FILTER && + _filter_type != RuntimeFilterType::IN_OR_BLOOM_FILTER) { + throw Exception(ErrorCode::INTERNAL_ERROR, + "init_bloom_filter meet invalid input type {}", int(_filter_type)); + } return _context->bloom_filter_func->init_with_cardinality(build_bf_cardinality); } @@ -391,7 +394,9 @@ class RuntimePredicateWrapper { BloomFilterFuncBase* get_bloomfilter() const { return _context->bloom_filter_func.get(); } void insert_fixed_len(const vectorized::ColumnPtr& column, size_t start) { - DCHECK(!is_ignored()); + if (is_ignored()) { + throw Exception(ErrorCode::INTERNAL_ERROR, "insert_fixed_len meet ignored rf"); + } switch (_filter_type) { case RuntimeFilterType::IN_FILTER: { _context->hybrid_set->insert_fixed_len(column, start); @@ -918,7 +923,10 @@ class RuntimePredicateWrapper { return _context->bloom_filter_func->contain_null(); } if (_context->hybrid_set) { - DCHECK(get_real_type() == RuntimeFilterType::IN_FILTER); + if (get_real_type() != RuntimeFilterType::IN_FILTER) { + throw Exception(ErrorCode::INTERNAL_ERROR, "rf has hybrid_set but real type is {}", + int(get_real_type())); + } return _context->hybrid_set->contain_null(); } if (_context->minmax_func) { From 51d21c4b97b2992317609935a9dcf7abc834c124 Mon Sep 17 00:00:00 2001 From: zhengyu Date: Thu, 21 Nov 2024 11:35:22 +0800 Subject: [PATCH 007/110] [test](doc) add cancel-warm-up example in doris's doc to regression test (#43286) Signed-off-by: freemandealer --- .../cancel-warm-up.md.groovy | 42 +++++++++++++++++++ 1 file changed, 42 insertions(+) create mode 100644 regression-test/suites/doc/sql-manual/sql-statements/Utility-Statements/cancel-warm-up.md.groovy diff --git a/regression-test/suites/doc/sql-manual/sql-statements/Utility-Statements/cancel-warm-up.md.groovy b/regression-test/suites/doc/sql-manual/sql-statements/Utility-Statements/cancel-warm-up.md.groovy new file mode 100644 index 000000000000000..925195540f98eb7 --- /dev/null +++ b/regression-test/suites/doc/sql-manual/sql-statements/Utility-Statements/cancel-warm-up.md.groovy @@ -0,0 +1,42 @@ +// 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. + +import org.junit.jupiter.api.Assertions; + +suite("docs/sql-manual/sql-statements/Utility-Statements/cancel-warm-up.md") { + + if (!isCloudMode()) { + return + } + + def show = { + multi_sql """ + SHOW WARM UP JOB; + """ + } + + try { + show() + multi_sql """ + CANCEL WARM UP JOB WHERE id = 1; + """ + } catch (Throwable t) { + println("examples in docs/sql-manual/sql-statements/Utility-Statements/cancel-warm-up.md is expected to fail on bad job id") + } finally { + show() + } +} From b104deae2a119755234f05df8aa48fa843b861ee Mon Sep 17 00:00:00 2001 From: Pxl Date: Thu, 21 Nov 2024 14:01:13 +0800 Subject: [PATCH 008/110] [Bug](runtime-filter) fix wrong throw of contain null (#44348) ### What problem does this PR solve? fix wrong throw of contain null introduced by #https://github.com/apache/doris/issues/43780 --- be/src/exprs/bloom_filter_func.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/be/src/exprs/bloom_filter_func.h b/be/src/exprs/bloom_filter_func.h index b5204fa767d59e8..54ad75028104845 100644 --- a/be/src/exprs/bloom_filter_func.h +++ b/be/src/exprs/bloom_filter_func.h @@ -17,6 +17,8 @@ #pragma once +#include "common/exception.h" +#include "common/status.h" #include "exprs/block_bloom_filter.hpp" #include "exprs/runtime_filter.h" #include "olap/rowset/segment_v2/bloom_filter.h" // IWYU pragma: keep @@ -205,7 +207,7 @@ class BloomFilterFuncBase : public RuntimeFilterFuncBase { bool contain_null() const { if (!_bloom_filter) { - throw Status::InternalError("_bloom_filter is nullptr"); + throw Exception(ErrorCode::INTERNAL_ERROR, "_bloom_filter is nullptr"); } return _bloom_filter->contain_null(); } From 3f86819154a0ec03fc2386d831e8701575de8cd7 Mon Sep 17 00:00:00 2001 From: zhangstar333 Date: Thu, 21 Nov 2024 14:02:40 +0800 Subject: [PATCH 009/110] [Bug](function)fix json_object function check null nums error as return bool (#44321) ### What problem does this PR solve? Related PR: #https://github.com/apache/doris/pull/34591/files Problem Summary: before the SQL report error, as it error change the simd::count_zero_num function return value static as bool, so it run into the if check of (not_null_num < size). but actual it's maybe return int type. now: ``` mysql [(none)]>select json_object ( CONCAT('k',t.number%30926%3000 + 0),CONCAT('k',t.number%30926%3000 + 0,t.number%1000000) ) from numbers("number" = "2") t order by 1; +-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | json_object(concat('k', cast((((number % 30926) % 3000) + 0) as VARCHAR(65533))), concat('k', cast((((number % 30926) % 3000) + 0) as VARCHAR(65533)), cast((number % 1000000) as VARCHAR(65533))), '66') | +-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | {"k0":"k00"} | | {"k1":"k11"} | +-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ 2 rows in set (0.04 sec) ``` before: ``` mysql [(none)]>select json_object -> ( -> CONCAT('k',t.number%30926%3000 + 0),CONCAT('k',t.number%30926%3000 + 0,t.number%1000000) -> ) -> from numbers("number" = "2") t; ERROR 1105 (HY000): errCode = 2, detailMessage = (10.16.10.8)[INTERNAL_ERROR]function json_object can not input null value , JSON documents may not contain NULL member names. mysql [(none)]> ``` --- be/src/vec/functions/function_json.cpp | 6 +++--- .../sql_functions/json_function/test_query_json_object.out | 4 ++++ .../json_function/test_query_json_object.groovy | 2 ++ 3 files changed, 9 insertions(+), 3 deletions(-) diff --git a/be/src/vec/functions/function_json.cpp b/be/src/vec/functions/function_json.cpp index 6ab03dbbb7b51ca..c53b31d7ec69f6c 100644 --- a/be/src/vec/functions/function_json.cpp +++ b/be/src/vec/functions/function_json.cpp @@ -788,13 +788,13 @@ class FunctionJsonAlwaysNotNullable : public IFunction { for (int i = 0; i < args; i += 2) { const auto* null_map = nullmaps[i]; if (null_map) { - const bool not_null_num = + auto not_null_num = simd::count_zero_num((int8_t*)null_map->get_data().data(), size); if (not_null_num < size) { return Status::InternalError( "function {} can not input null value , JSON documents may not contain " - "NULL member names.", - name); + "NULL member names. input size is {}:{}", + name, size, not_null_num); } } } diff --git a/regression-test/data/query_p0/sql_functions/json_function/test_query_json_object.out b/regression-test/data/query_p0/sql_functions/json_function/test_query_json_object.out index 6dc88471573c5d4..13d30f6e75eaeac 100644 --- a/regression-test/data/query_p0/sql_functions/json_function/test_query_json_object.out +++ b/regression-test/data/query_p0/sql_functions/json_function/test_query_json_object.out @@ -6,3 +6,7 @@ {"k0":4,"k1":null,"k2":null,"k3":"test","k4":"2022-01-01 11:11:11","k5":null,"k6":"k6"} {"k0":5,"k1":1,"k2":true,"k3":"test","k4":"2022-01-01 11:11:11","k5":null,"k6":"k6"} +-- !sql2 -- +{"k0":"k00"} +{"k1":"k11"} + diff --git a/regression-test/suites/query_p0/sql_functions/json_function/test_query_json_object.groovy b/regression-test/suites/query_p0/sql_functions/json_function/test_query_json_object.groovy index c3b1e35b5e663a8..2ee0c64276c6ad8 100644 --- a/regression-test/suites/query_p0/sql_functions/json_function/test_query_json_object.groovy +++ b/regression-test/suites/query_p0/sql_functions/json_function/test_query_json_object.groovy @@ -45,5 +45,7 @@ suite("test_query_json_object", "query") { sql """select k0,json_object(k3,123) from ${tableName} order by k0;""" exception "function json_object can not input null value , JSON documents may not contain NULL member names." } + + qt_sql2 """select json_object ( CONCAT('k',t.number%30926%3000 + 0),CONCAT('k',t.number%30926%3000 + 0,t.number%1000000) ) from numbers("number" = "2") t order by 1;""" sql "DROP TABLE ${tableName};" } From 4c58d62b2af82337dcb34af2a92bf53026eeae71 Mon Sep 17 00:00:00 2001 From: nsn_huang <38585669+nsnhuang@users.noreply.github.com> Date: Thu, 21 Nov 2024 14:04:07 +0800 Subject: [PATCH 010/110] [fix](config) fe config sync_image_timeout_second should not be masterOnly (#43954) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What problem does this PR solve? sync_image_timeout_second is for the follower,should not be masterOnly Co-authored-by: huangwenbo04 --- fe/fe-common/src/main/java/org/apache/doris/common/Config.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index efc2d11a092e3b9..689c85940dfc394 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -2673,7 +2673,7 @@ public class Config extends ConfigBase { }) public static boolean ignore_unknown_metadata_module = false; - @ConfField(mutable = true, masterOnly = true, description = { + @ConfField(mutable = true, description = { "从主节点同步image文件的超时时间,用户可根据${meta_dir}/image文件夹下面的image文件大小和节点间的网络环境调整," + "单位为秒,默认值300", "The timeout for FE Follower/Observer synchronizing an image file from the FE Master, can be adjusted by " From babd6cea6ea7791288d246f240a8750386fb2f38 Mon Sep 17 00:00:00 2001 From: Kaijie Chen Date: Thu, 21 Nov 2024 14:07:36 +0800 Subject: [PATCH 011/110] [fix](move-memtable) immediately return error when close wait failed (#44344) Problem Summary: #38003 introduced a problem where the last sink node could report success even when close wait timeout, which may cause data loss. Previously we made that change hoping to tolerate minority replica failure in this step. However, it turns out the last sink node could miss tablet reports from downstreams in case of close wait failure. This PR fixes the problem by return the close_wait error immediately. The most common error in close wait is timeout, and it should not be fault tolerant on a replica basis anyways. --- be/src/vec/sink/writer/vtablet_writer_v2.cpp | 7 ++++--- be/src/vec/sink/writer/vtablet_writer_v2.h | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/be/src/vec/sink/writer/vtablet_writer_v2.cpp b/be/src/vec/sink/writer/vtablet_writer_v2.cpp index 96dfd85d297208a..3dc58be3bcde885 100644 --- a/be/src/vec/sink/writer/vtablet_writer_v2.cpp +++ b/be/src/vec/sink/writer/vtablet_writer_v2.cpp @@ -606,7 +606,7 @@ Status VTabletWriterV2::close(Status exec_status) { // close_wait on all non-incremental streams, even if this is not the last sink. // because some per-instance data structures are now shared among all sinks // due to sharing delta writers and load stream stubs. - _close_wait(false); + RETURN_IF_ERROR(_close_wait(false)); // send CLOSE_LOAD on all incremental streams if this is the last sink. // this must happen after all non-incremental streams are closed, @@ -616,7 +616,7 @@ Status VTabletWriterV2::close(Status exec_status) { } // close_wait on all incremental streams, even if this is not the last sink. - _close_wait(true); + RETURN_IF_ERROR(_close_wait(true)); // calculate and submit commit info if (is_last_sink) { @@ -665,7 +665,7 @@ Status VTabletWriterV2::close(Status exec_status) { return status; } -void VTabletWriterV2::_close_wait(bool incremental) { +Status VTabletWriterV2::_close_wait(bool incremental) { SCOPED_TIMER(_close_load_timer); auto st = _load_stream_map->for_each_st( [this, incremental](int64_t dst_id, LoadStreamStubs& streams) -> Status { @@ -690,6 +690,7 @@ void VTabletWriterV2::_close_wait(bool incremental) { if (!st.ok()) { LOG(WARNING) << "close_wait failed: " << st << ", load_id=" << print_id(_load_id); } + return st; } void VTabletWriterV2::_calc_tablets_to_commit() { diff --git a/be/src/vec/sink/writer/vtablet_writer_v2.h b/be/src/vec/sink/writer/vtablet_writer_v2.h index 52cae53a0b5d0a6..46a3974bba8aa00 100644 --- a/be/src/vec/sink/writer/vtablet_writer_v2.h +++ b/be/src/vec/sink/writer/vtablet_writer_v2.h @@ -148,7 +148,7 @@ class VTabletWriterV2 final : public AsyncResultWriter { void _calc_tablets_to_commit(); - void _close_wait(bool incremental); + Status _close_wait(bool incremental); void _cancel(Status status); From 940fc4e25285234093b78d652394f0303ca493b7 Mon Sep 17 00:00:00 2001 From: Gabriel Date: Thu, 21 Nov 2024 14:42:55 +0800 Subject: [PATCH 012/110] [profile](runtime filter) Add necessary metrics in runtime filter (#44367) Currently, we lost wait time and local merge time for each runtime filter. This PR complete some necessary metrics. --- be/src/exprs/runtime_filter.cpp | 61 +++++++++---------- be/src/exprs/runtime_filter.h | 9 +-- .../local_exchange_sink_operator.cpp | 3 + .../local_exchange_sink_operator.h | 2 + be/src/runtime/fragment_mgr.cpp | 4 +- be/src/runtime/runtime_filter_mgr.cpp | 30 +-------- be/src/runtime/runtime_filter_mgr.h | 6 +- gensrc/proto/internal_service.proto | 2 + 8 files changed, 50 insertions(+), 67 deletions(-) diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index 2ca6493dc31c861..24333360ff62540 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -1001,20 +1001,20 @@ void IRuntimeFilter::insert_batch(const vectorized::ColumnPtr column, size_t sta Status IRuntimeFilter::publish(RuntimeState* state, bool publish_local) { DCHECK(is_producer()); - auto send_to_remote_targets = [&](IRuntimeFilter* filter) { + auto send_to_remote_targets = [&](IRuntimeFilter* filter, uint64_t local_merge_time) { TNetworkAddress addr; DCHECK(_state != nullptr); RETURN_IF_ERROR(_state->global_runtime_filter_mgr()->get_merge_addr(&addr)); - return filter->push_to_remote(state, &addr); + return filter->push_to_remote(state, &addr, local_merge_time); }; - auto send_to_local_targets = [&](std::shared_ptr wrapper, - bool global) { + auto send_to_local_targets = [&](std::shared_ptr wrapper, bool global, + uint64_t local_merge_time = 0) { std::vector> filters = global ? _state->global_runtime_filter_mgr()->get_consume_filters(_filter_id) : _state->local_runtime_filter_mgr()->get_consume_filters(_filter_id); for (auto filter : filters) { filter->_wrapper = wrapper; - filter->update_runtime_filter_type_to_profile(); + filter->update_runtime_filter_type_to_profile(local_merge_time); filter->signal(); } return Status::OK(); @@ -1024,15 +1024,20 @@ Status IRuntimeFilter::publish(RuntimeState* state, bool publish_local) { LocalMergeFilters* local_merge_filters = nullptr; RETURN_IF_ERROR(_state->global_runtime_filter_mgr()->get_local_merge_producer_filters( _filter_id, &local_merge_filters)); + local_merge_filters->merge_watcher.start(); std::lock_guard l(*local_merge_filters->lock); RETURN_IF_ERROR(local_merge_filters->filters[0]->merge_from(_wrapper.get())); local_merge_filters->merge_time--; + local_merge_filters->merge_watcher.stop(); if (local_merge_filters->merge_time == 0) { if (_has_local_target) { - RETURN_IF_ERROR( - send_to_local_targets(local_merge_filters->filters[0]->_wrapper, true)); + RETURN_IF_ERROR(send_to_local_targets( + local_merge_filters->filters[0]->_wrapper, true, + local_merge_filters->merge_watcher.elapsed_time())); } else { - RETURN_IF_ERROR(send_to_remote_targets(local_merge_filters->filters[0].get())); + RETURN_IF_ERROR(send_to_remote_targets( + local_merge_filters->filters[0].get(), + local_merge_filters->merge_watcher.elapsed_time())); } } } @@ -1046,7 +1051,7 @@ Status IRuntimeFilter::publish(RuntimeState* state, bool publish_local) { RETURN_IF_ERROR(send_to_local_targets(_wrapper, false)); } else if (!publish_local) { if (_is_broadcast_join || _state->get_query_ctx()->be_exec_version() < USE_NEW_SERDE) { - RETURN_IF_ERROR(send_to_remote_targets(this)); + RETURN_IF_ERROR(send_to_remote_targets(this, 0)); } else { RETURN_IF_ERROR(do_merge()); } @@ -1176,7 +1181,8 @@ Status IRuntimeFilter::send_filter_size(RuntimeState* state, uint64_t local_filt return Status::OK(); } -Status IRuntimeFilter::push_to_remote(RuntimeState* state, const TNetworkAddress* addr) { +Status IRuntimeFilter::push_to_remote(RuntimeState* state, const TNetworkAddress* addr, + uint64_t local_merge_time) { DCHECK(is_producer()); std::shared_ptr stub( _state->get_query_ctx()->exec_env()->brpc_internal_client_cache()->get_client(*addr)); @@ -1205,6 +1211,7 @@ Status IRuntimeFilter::push_to_remote(RuntimeState* state, const TNetworkAddress pfragment_instance_id->set_lo((int64_t)this); merge_filter_request->set_filter_id(_filter_id); + merge_filter_request->set_local_merge_time(local_merge_time); auto column_type = _wrapper->column_type(); RETURN_IF_CATCH_EXCEPTION(merge_filter_request->set_column_type(to_proto(column_type))); @@ -1244,9 +1251,9 @@ Status IRuntimeFilter::get_push_expr_ctxs(std::listadd_info_string("Info", formatted_state()); // The runtime filter is pushed down, adding filtering information. - auto* expr_filtered_rows_counter = ADD_COUNTER(_profile, "expr_filtered_rows", TUnit::UNIT); - auto* expr_input_rows_counter = ADD_COUNTER(_profile, "expr_input_rows", TUnit::UNIT); - auto* always_true_counter = ADD_COUNTER(_profile, "always_true_pass_rows", TUnit::UNIT); + auto* expr_filtered_rows_counter = ADD_COUNTER(_profile, "ExprFilteredRows", TUnit::UNIT); + auto* expr_input_rows_counter = ADD_COUNTER(_profile, "ExprInputRows", TUnit::UNIT); + auto* always_true_counter = ADD_COUNTER(_profile, "AlwaysTruePassRows", TUnit::UNIT); for (auto i = origin_size; i < push_exprs.size(); i++) { push_exprs[i]->attach_profile_counter(expr_filtered_rows_counter, expr_input_rows_counter, always_true_counter); @@ -1266,6 +1273,7 @@ void IRuntimeFilter::update_state() { // In pipelineX, runtime filters will be ready or timeout before open phase. if (expected == RuntimeFilterState::NOT_READY) { DCHECK(MonotonicMillis() - registration_time_ >= wait_times_ms); + COUNTER_SET(_wait_timer, MonotonicMillis() - registration_time_); _rf_state_atomic = RuntimeFilterState::TIME_OUT; } } @@ -1284,6 +1292,7 @@ PrimitiveType IRuntimeFilter::column_type() const { void IRuntimeFilter::signal() { DCHECK(is_consumer()); + COUNTER_SET(_wait_timer, MonotonicMillis() - registration_time_); _rf_state_atomic.store(RuntimeFilterState::READY); if (!_filter_timer.empty()) { for (auto& timer : _filter_timer) { @@ -1523,11 +1532,14 @@ void IRuntimeFilter::init_profile(RuntimeProfile* parent_profile) { _profile_init = true; parent_profile->add_child(_profile.get(), true, nullptr); _profile->add_info_string("Info", formatted_state()); + _wait_timer = ADD_TIMER(_profile, "WaitTime"); } } -void IRuntimeFilter::update_runtime_filter_type_to_profile() { +void IRuntimeFilter::update_runtime_filter_type_to_profile(uint64_t local_merge_time) { _profile->add_info_string("RealRuntimeFilterType", to_string(_wrapper->get_real_type())); + _profile->add_info_string("LocalMergeTime", + std::to_string(local_merge_time / 1000000000.0) + " s"); } std::string IRuntimeFilter::debug_string() const { @@ -1864,24 +1876,9 @@ bool IRuntimeFilter::need_sync_filter_size() { _wrapper->get_build_bf_cardinality() && !_is_broadcast_join; } -Status IRuntimeFilter::update_filter(const UpdateRuntimeFilterParams* param) { - _profile->add_info_string("MergeTime", std::to_string(param->request->merge_time()) + " ms"); - - if (param->request->has_ignored() && param->request->ignored()) { - set_ignored(); - } else { - std::unique_ptr wrapper; - RETURN_IF_ERROR(IRuntimeFilter::create_wrapper(param, &wrapper)); - RETURN_IF_ERROR(_wrapper->merge(wrapper.get())); - update_runtime_filter_type_to_profile(); - } - this->signal(); - - return Status::OK(); -} - void IRuntimeFilter::update_filter(std::shared_ptr wrapper, - int64_t merge_time, int64_t start_apply) { + int64_t merge_time, int64_t start_apply, + uint64_t local_merge_time) { _profile->add_info_string("UpdateTime", std::to_string(MonotonicMillis() - start_apply) + " ms"); _profile->add_info_string("MergeTime", std::to_string(merge_time) + " ms"); @@ -1891,7 +1888,7 @@ void IRuntimeFilter::update_filter(std::shared_ptr wrap wrapper->_column_return_type = _wrapper->_column_return_type; } _wrapper = wrapper; - update_runtime_filter_type_to_profile(); + update_runtime_filter_type_to_profile(local_merge_time); signal(); } diff --git a/be/src/exprs/runtime_filter.h b/be/src/exprs/runtime_filter.h index 9d9021a747ba7b6..50ee52865be6d6f 100644 --- a/be/src/exprs/runtime_filter.h +++ b/be/src/exprs/runtime_filter.h @@ -278,9 +278,8 @@ class IRuntimeFilter { std::shared_ptr* wrapper); Status change_to_bloom_filter(); Status init_bloom_filter(const size_t build_bf_cardinality); - Status update_filter(const UpdateRuntimeFilterParams* param); void update_filter(std::shared_ptr filter_wrapper, int64_t merge_time, - int64_t start_apply); + int64_t start_apply, uint64_t local_merge_time); void set_ignored(); @@ -291,13 +290,14 @@ class IRuntimeFilter { bool need_sync_filter_size(); // async push runtimefilter to remote node - Status push_to_remote(RuntimeState* state, const TNetworkAddress* addr); + Status push_to_remote(RuntimeState* state, const TNetworkAddress* addr, + uint64_t local_merge_time); void init_profile(RuntimeProfile* parent_profile); std::string debug_string() const; - void update_runtime_filter_type_to_profile(); + void update_runtime_filter_type_to_profile(uint64_t local_merge_time); int filter_id() const { return _filter_id; } @@ -415,6 +415,7 @@ class IRuntimeFilter { // parent profile // only effect on consumer std::unique_ptr _profile; + RuntimeProfile::Counter* _wait_timer = nullptr; std::vector> _filter_timer; diff --git a/be/src/pipeline/local_exchange/local_exchange_sink_operator.cpp b/be/src/pipeline/local_exchange/local_exchange_sink_operator.cpp index 0369cf75834ce90..22007a4b220348b 100644 --- a/be/src/pipeline/local_exchange/local_exchange_sink_operator.cpp +++ b/be/src/pipeline/local_exchange/local_exchange_sink_operator.cpp @@ -91,6 +91,9 @@ Status LocalExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo "UseGlobalShuffle", std::to_string(_parent->cast()._use_global_shuffle)); } + _profile->add_info_string( + "PartitionExprsSize", + std::to_string(_parent->cast()._partitioned_exprs_num)); _channel_id = info.task_idx; return Status::OK(); } diff --git a/be/src/pipeline/local_exchange/local_exchange_sink_operator.h b/be/src/pipeline/local_exchange/local_exchange_sink_operator.h index 4c4a400c2bde3bc..435f7a410a4ca6e 100644 --- a/be/src/pipeline/local_exchange/local_exchange_sink_operator.h +++ b/be/src/pipeline/local_exchange/local_exchange_sink_operator.h @@ -91,6 +91,7 @@ class LocalExchangeSinkOperatorX final : public DataSinkOperatorX& _texprs; + const size_t _partitioned_exprs_num; std::unique_ptr _partitioner; const std::map _bucket_seq_to_instance_idx; std::vector> _shuffle_idx_to_instance_idx; diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index c4f633d84aa2789..5a8ea2377aa80fc 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -1245,7 +1245,9 @@ Status FragmentMgr::apply_filterv2(const PPublishFilterRequestV2* request, RETURN_IF_ERROR(IRuntimeFilter::create_wrapper(¶ms, &filter_wrapper)); std::ranges::for_each(filters, [&](auto& filter) { - filter->update_filter(filter_wrapper, request->merge_time(), start_apply); + filter->update_filter( + filter_wrapper, request->merge_time(), start_apply, + request->has_local_merge_time() ? request->local_merge_time() : 0); }); } diff --git a/be/src/runtime/runtime_filter_mgr.cpp b/be/src/runtime/runtime_filter_mgr.cpp index 70ce8add789d13a..bb100fcbb42ec5f 100644 --- a/be/src/runtime/runtime_filter_mgr.cpp +++ b/be/src/runtime/runtime_filter_mgr.cpp @@ -201,33 +201,6 @@ Status RuntimeFilterMgr::register_producer_filter(const TRuntimeFilterDesc& desc return Status::OK(); } -Status RuntimeFilterMgr::update_filter(const PPublishFilterRequest* request, - butil::IOBufAsZeroCopyInputStream* data) { - SCOPED_CONSUME_MEM_TRACKER(_tracker.get()); - UpdateRuntimeFilterParams params(request, data); - int filter_id = request->filter_id(); - std::vector> filters; - // The code is organized for upgrade compatibility to prevent infinite waiting - // old way update filter the code should be deleted after the upgrade is complete. - { - std::lock_guard l(_lock); - auto iter = _consumer_map.find(filter_id); - if (iter == _consumer_map.end()) { - return Status::InternalError("update_filter meet unknown filter: {}, role: CONSUMER.", - filter_id); - } - for (auto& holder : iter->second) { - filters.emplace_back(holder.filter); - } - iter->second.clear(); - } - for (auto filter : filters) { - RETURN_IF_ERROR(filter->update_filter(¶ms)); - } - - return Status::OK(); -} - void RuntimeFilterMgr::set_runtime_filter_params( const TRuntimeFilterParams& runtime_filter_params) { std::lock_guard l(_lock); @@ -442,6 +415,8 @@ Status RuntimeFilterMergeControllerEntity::merge(std::weak_ptr que DCHECK_LE(merged_size, cnt_val->producer_size); cnt_val->merge_time += (MonotonicMillis() - start_merge); merge_time = cnt_val->merge_time; + cnt_val->local_merge_time += + request->has_local_merge_time() ? request->local_merge_time() : 0; } if (merged_size == cnt_val->producer_size) { @@ -481,6 +456,7 @@ Status RuntimeFilterMergeControllerEntity::merge(std::weak_ptr que closure->request_->set_filter_id(request->filter_id()); closure->request_->set_merge_time(merge_time); + closure->request_->set_local_merge_time(cnt_val->local_merge_time); *closure->request_->mutable_query_id() = request->query_id(); if (has_attachment) { closure->cntl_->request_attachment().append(request_attachment); diff --git a/be/src/runtime/runtime_filter_mgr.h b/be/src/runtime/runtime_filter_mgr.h index 83c526c31a95694..0a6f8318feaba03 100644 --- a/be/src/runtime/runtime_filter_mgr.h +++ b/be/src/runtime/runtime_filter_mgr.h @@ -34,6 +34,7 @@ #include "common/object_pool.h" #include "common/status.h" +#include "util/stopwatch.hpp" #include "util/uid_util.h" namespace butil { @@ -60,6 +61,7 @@ struct LocalMergeFilters { int merge_size_times = 0; uint64_t local_merged_size = 0; std::vector> filters; + MonotonicStopWatch merge_watcher; }; /// producer: @@ -113,9 +115,6 @@ class RuntimeFilterMgr { bool build_bf_exactly = false); // update filter by remote - Status update_filter(const PPublishFilterRequest* request, - butil::IOBufAsZeroCopyInputStream* data); - void set_runtime_filter_params(const TRuntimeFilterParams& runtime_filter_params); Status get_merge_addr(TNetworkAddress* addr); @@ -189,6 +188,7 @@ class RuntimeFilterMergeControllerEntity { std::unordered_set arrive_id; std::vector source_addrs; std::shared_ptr pool; + uint64_t local_merge_time = 0; }; private: diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto index 6a9339459332e15..8db8bab16cd499b 100644 --- a/gensrc/proto/internal_service.proto +++ b/gensrc/proto/internal_service.proto @@ -594,6 +594,7 @@ message PMergeFilterRequest { optional PColumnType column_type = 10; optional bool contain_null = 11; optional bool ignored = 12; + optional uint64 local_merge_time = 13; }; message PMergeFilterResponse { @@ -629,6 +630,7 @@ message PPublishFilterRequestV2 { optional bool contain_null = 10; optional bool ignored = 11; repeated int32 fragment_ids = 12; + optional uint64 local_merge_time = 13; }; message PPublishFilterResponse { From 738d0bf1cd774946dca7c90d4839cf4579e07fc6 Mon Sep 17 00:00:00 2001 From: LiBinfeng Date: Thu, 21 Nov 2024 14:56:03 +0800 Subject: [PATCH 013/110] [feat](Nereids) support alter job status commands (#44279) --- .../org/apache/doris/nereids/DorisParser.g4 | 13 +-- .../nereids/parser/LogicalPlanBuilder.java | 52 ++++++++++ .../doris/nereids/trees/plans/PlanType.java | 4 + .../plans/commands/AlterJobStatusCommand.java | 98 ++++++++++++++++++ .../plans/commands/CancelJobTaskCommand.java | 99 +++++++++++++++++++ .../trees/plans/commands/DropJobCommand.java | 51 ++++++++++ .../trees/plans/commands/PauseJobCommand.java | 50 ++++++++++ .../plans/commands/ResumeJobCommand.java | 50 ++++++++++ .../trees/plans/visitor/CommandVisitor.java | 25 +++++ 9 files changed, 433 insertions(+), 9 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobStatusCommand.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelJobTaskCommand.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropJobCommand.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/PauseJobCommand.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ResumeJobCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 97932c3bb2afd91..cc7ade25fb2b5fc 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -76,7 +76,6 @@ unsupportedStatement | unsupportedAdminStatement | unsupportedTransactionStatement | unsupportedCancelStatement - | unsupportedJobStatement | unsupportedCleanStatement | unsupportedRefreshStatement | unsupportedLoadStatement @@ -117,6 +116,10 @@ supportedJobStatement (AT (atTime=STRING_LITERAL | CURRENT_TIMESTAMP))) commentSpec? DO supportedDmlStatement #createScheduledJob + | PAUSE JOB wildWhere? #pauseJob + | DROP JOB (IF EXISTS)? wildWhere? #dropJob + | RESUME JOB wildWhere? #resumeJob + | CANCEL TASK wildWhere? #cancelJobTask ; constraintStatement : ALTER TABLE table=multipartIdentifier @@ -437,14 +440,6 @@ unsupportedCleanStatement | CLEAN ALL QUERY STATS #cleanAllQueryStats ; -unsupportedJobStatement - - : PAUSE JOB wildWhere? #pauseJob - | DROP JOB (IF EXISTS)? wildWhere? #dropJob - | RESUME JOB wildWhere? #resumeJob - | CANCEL TASK wildWhere? #cancelJobTask - ; - unsupportedCancelStatement : CANCEL LOAD ((FROM | IN) database=identifier)? wildWhere? #cancelLoad | CANCEL EXPORT ((FROM | IN) database=identifier)? wildWhere? #cancelExport diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 36b25e474df7459..00cc011ef23c15a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -421,6 +421,7 @@ import org.apache.doris.nereids.trees.plans.commands.AlterStorageVaultCommand; import org.apache.doris.nereids.trees.plans.commands.AlterViewCommand; import org.apache.doris.nereids.trees.plans.commands.CallCommand; +import org.apache.doris.nereids.trees.plans.commands.CancelJobTaskCommand; import org.apache.doris.nereids.trees.plans.commands.CancelMTMVTaskCommand; import org.apache.doris.nereids.trees.plans.commands.Command; import org.apache.doris.nereids.trees.plans.commands.Constraint; @@ -436,6 +437,7 @@ import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand; import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand.IdType; import org.apache.doris.nereids.trees.plans.commands.DropConstraintCommand; +import org.apache.doris.nereids.trees.plans.commands.DropJobCommand; import org.apache.doris.nereids.trees.plans.commands.DropMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.DropProcedureCommand; import org.apache.doris.nereids.trees.plans.commands.DropRoleCommand; @@ -444,12 +446,14 @@ import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; import org.apache.doris.nereids.trees.plans.commands.ExportCommand; import org.apache.doris.nereids.trees.plans.commands.LoadCommand; +import org.apache.doris.nereids.trees.plans.commands.PauseJobCommand; import org.apache.doris.nereids.trees.plans.commands.PauseMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.RecoverDatabaseCommand; import org.apache.doris.nereids.trees.plans.commands.RecoverPartitionCommand; import org.apache.doris.nereids.trees.plans.commands.RecoverTableCommand; import org.apache.doris.nereids.trees.plans.commands.RefreshMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.ReplayCommand; +import org.apache.doris.nereids.trees.plans.commands.ResumeJobCommand; import org.apache.doris.nereids.trees.plans.commands.ResumeMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.SetDefaultStorageVaultCommand; import org.apache.doris.nereids.trees.plans.commands.SetOptionsCommand; @@ -676,6 +680,43 @@ public LogicalPlan visitCreateScheduledJob(DorisParser.CreateScheduledJobContext return new CreateJobCommand(createJobInfo); } + @Override + public LogicalPlan visitPauseJob(DorisParser.PauseJobContext ctx) { + Expression wildWhere = null; + if (ctx.wildWhere() != null) { + wildWhere = getWildWhere(ctx.wildWhere()); + } + return new PauseJobCommand(wildWhere); + } + + @Override + public LogicalPlan visitDropJob(DorisParser.DropJobContext ctx) { + Expression wildWhere = null; + if (ctx.wildWhere() != null) { + wildWhere = getWildWhere(ctx.wildWhere()); + } + boolean ifExists = ctx.EXISTS() != null; + return new DropJobCommand(wildWhere, ifExists); + } + + @Override + public LogicalPlan visitResumeJob(DorisParser.ResumeJobContext ctx) { + Expression wildWhere = null; + if (ctx.wildWhere() != null) { + wildWhere = getWildWhere(ctx.wildWhere()); + } + return new ResumeJobCommand(wildWhere); + } + + @Override + public LogicalPlan visitCancelJobTask(DorisParser.CancelJobTaskContext ctx) { + Expression wildWhere = null; + if (ctx.wildWhere() != null) { + wildWhere = getWildWhere(ctx.wildWhere()); + } + return new CancelJobTaskCommand(wildWhere); + } + @Override public String visitCommentSpec(DorisParser.CommentSpecContext ctx) { String commentSpec = ctx == null ? "''" : ctx.STRING_LITERAL().getText(); @@ -4138,6 +4179,17 @@ public LogicalPlan visitShowVariables(ShowVariablesContext ctx) { } } + private Expression getWildWhere(DorisParser.WildWhereContext ctx) { + if (ctx.LIKE() != null) { + String pattern = stripQuotes(ctx.STRING_LITERAL().getText()); + return new Like(new UnboundSlot("ProcedureName"), new StringLiteral(pattern)); + } else if (ctx.WHERE() != null) { + return getExpression(ctx.expression()); + } else { + throw new AnalysisException("Wild where should contain like or where " + ctx.getText()); + } + } + @Override public ShowViewCommand visitShowView(ShowViewContext ctx) { List tableNameParts = visitMultipartIdentifier(ctx.tableName); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index e040d18b82f4fae..ccfb505624b554c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -145,6 +145,10 @@ public enum PlanType { UPDATE_COMMAND, CREATE_MTMV_COMMAND, CREATE_JOB_COMMAND, + PAUSE_JOB_COMMAND, + CANCEL_JOB_COMMAND, + DROP_JOB_COMMAND, + RESUME_JOB_COMMAND, ALTER_MTMV_COMMAND, ADD_CONSTRAINT_COMMAND, DROP_CONSTRAINT_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobStatusCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobStatusCommand.java new file mode 100644 index 000000000000000..c3aafa8a5fe0c5a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobStatusCommand.java @@ -0,0 +1,98 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.StmtType; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.trees.expressions.EqualTo; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.literal.StringLikeLiteral; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; + +/** + * base class for all drop commands + */ +public abstract class AlterJobStatusCommand extends Command implements ForwardWithSync { + // exclude job name prefix, which is used by inner job + private static final String excludeJobNamePrefix = "inner_"; + private final Expression wildWhere; + private String jobName; + + public AlterJobStatusCommand(PlanType type, Expression wildWhere) { + super(type); + this.wildWhere = wildWhere; + } + + public String getJobName() { + return jobName; + } + + @Override + public StmtType stmtType() { + return StmtType.ALTER; + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + validate(); + doRun(ctx, executor); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitAlterJobStatusCommand(this, context); + } + + private void validate() throws Exception { + if (!(wildWhere instanceof EqualTo)) { + throw new AnalysisException("Alter job status only support equal condition, but not: " + wildWhere.toSql()); + } + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + Expression left = ((EqualTo) wildWhere).left(); + Expression right = ((EqualTo) wildWhere).right(); + if (!(left instanceof UnboundSlot && ((UnboundSlot) left).getName().equalsIgnoreCase("jobName"))) { + throw new AnalysisException("Current not support left child of where: " + left); + } + if (!(right instanceof StringLikeLiteral)) { + throw new AnalysisException("Value must is string"); + } + + if (Strings.isNullOrEmpty(((StringLikeLiteral) right).getStringValue())) { + throw new AnalysisException("Value can't is null"); + } + this.jobName = ((StringLikeLiteral) right).getStringValue(); + if (jobName.startsWith(excludeJobNamePrefix)) { + throw new AnalysisException("Can't alter inner job status"); + } + } + + public abstract void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception; + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelJobTaskCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelJobTaskCommand.java new file mode 100644 index 000000000000000..a9ea241e3b63eca --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CancelJobTaskCommand.java @@ -0,0 +1,99 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.trees.expressions.And; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.literal.LargeIntLiteral; +import org.apache.doris.nereids.trees.expressions.literal.StringLikeLiteral; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +/** + * base class for all drop commands + */ +public class CancelJobTaskCommand extends Command implements ForwardWithSync { + private static final String jobNameKey = "jobName"; + + private static final String taskIdKey = "taskId"; + + private String jobName; + + private Long taskId; + + private Expression expr; + + public CancelJobTaskCommand(Expression expr) { + super(PlanType.CANCEL_JOB_COMMAND); + this.expr = expr; + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCancelTaskCommand(this, context); + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + validate(); + doRun(ctx); + } + + private void validate() throws AnalysisException { + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + if (!(expr instanceof And)) { + throw new AnalysisException("Only allow compound predicate with operator AND"); + } + if (!(expr.child(0).child(0) instanceof UnboundSlot) + && jobNameKey.equals(((UnboundSlot) expr.child(0).child(0)).getName())) { + throw new AnalysisException("Current not support " + ((UnboundSlot) expr.child(0).child(0)).getName()); + } + + if (!(expr.child(0).child(1) instanceof StringLikeLiteral)) { + throw new AnalysisException("JobName value must is string"); + } + this.jobName = ((StringLikeLiteral) expr.child(0).child(1)).getStringValue(); + String taskIdInput = ((StringLikeLiteral) expr.child(1).child(0)).getStringValue(); + if (!taskIdKey.equalsIgnoreCase(taskIdInput)) { + throw new AnalysisException("Current not support " + taskIdInput); + } + if (!(expr.child(1).child(1) instanceof LargeIntLiteral)) { + throw new AnalysisException("task id value must is large int"); + } + this.taskId = ((LargeIntLiteral) expr.child(1).child(1)).getLongValue(); + } + + public void doRun(ConnectContext ctx) throws Exception { + try { + ctx.getEnv().getJobManager().cancelTaskById(jobName, taskId); + } catch (Exception e) { + throw new DdlException(e.getMessage()); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropJobCommand.java new file mode 100644 index 000000000000000..69c00a0b0846948 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropJobCommand.java @@ -0,0 +1,51 @@ +// 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.nereids.trees.plans.commands; + +import org.apache.doris.analysis.StmtType; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +/** + * base class for all drop commands + */ +public class DropJobCommand extends AlterJobStatusCommand implements ForwardWithSync { + private final boolean ifExists; + + public DropJobCommand(Expression wildWhere, boolean ifExists) { + super(PlanType.DROP_JOB_COMMAND, wildWhere); + this.ifExists = ifExists; + } + + @Override + public StmtType stmtType() { + return StmtType.DROP; + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitDropJobCommand(this, context); + } + + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + ctx.getEnv().getJobManager().unregisterJob(super.getJobName(), ifExists); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/PauseJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/PauseJobCommand.java new file mode 100644 index 000000000000000..2954c79b074dfe8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/PauseJobCommand.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.nereids.trees.plans.commands; + +import org.apache.doris.analysis.StmtType; +import org.apache.doris.job.common.JobStatus; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +/** + * pause job + */ +public class PauseJobCommand extends AlterJobStatusCommand implements ForwardWithSync { + public PauseJobCommand(Expression wildWhere) { + super(PlanType.PAUSE_JOB_COMMAND, wildWhere); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitPauseJobCommand(this, context); + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + ctx.getEnv().getJobManager().alterJobStatus(super.getJobName(), JobStatus.PAUSED); + } + + @Override + public StmtType stmtType() { + return StmtType.PAUSE; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ResumeJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ResumeJobCommand.java new file mode 100644 index 000000000000000..88ee9fe0774186f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ResumeJobCommand.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.nereids.trees.plans.commands; + +import org.apache.doris.analysis.StmtType; +import org.apache.doris.job.common.JobStatus; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +/** + * pause job + */ +public class ResumeJobCommand extends AlterJobStatusCommand implements ForwardWithSync { + public ResumeJobCommand(Expression wildWhere) { + super(PlanType.RESUME_JOB_COMMAND, wildWhere); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitResumeJobCommand(this, context); + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + ctx.getEnv().getJobManager().alterJobStatus(super.getJobName(), JobStatus.RUNNING); + } + + @Override + public StmtType stmtType() { + return StmtType.PAUSE; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 9bffb5a376e45e8..8e08f75aebbd7ae 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -18,10 +18,12 @@ package org.apache.doris.nereids.trees.plans.visitor; import org.apache.doris.nereids.trees.plans.commands.AddConstraintCommand; +import org.apache.doris.nereids.trees.plans.commands.AlterJobStatusCommand; import org.apache.doris.nereids.trees.plans.commands.AlterMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.AlterRoleCommand; import org.apache.doris.nereids.trees.plans.commands.AlterViewCommand; import org.apache.doris.nereids.trees.plans.commands.CallCommand; +import org.apache.doris.nereids.trees.plans.commands.CancelJobTaskCommand; import org.apache.doris.nereids.trees.plans.commands.CancelMTMVTaskCommand; import org.apache.doris.nereids.trees.plans.commands.Command; import org.apache.doris.nereids.trees.plans.commands.CreateJobCommand; @@ -35,6 +37,7 @@ import org.apache.doris.nereids.trees.plans.commands.DeleteFromUsingCommand; import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand; import org.apache.doris.nereids.trees.plans.commands.DropConstraintCommand; +import org.apache.doris.nereids.trees.plans.commands.DropJobCommand; import org.apache.doris.nereids.trees.plans.commands.DropMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.DropProcedureCommand; import org.apache.doris.nereids.trees.plans.commands.DropRoleCommand; @@ -42,12 +45,14 @@ import org.apache.doris.nereids.trees.plans.commands.ExplainCommand; import org.apache.doris.nereids.trees.plans.commands.ExportCommand; import org.apache.doris.nereids.trees.plans.commands.LoadCommand; +import org.apache.doris.nereids.trees.plans.commands.PauseJobCommand; import org.apache.doris.nereids.trees.plans.commands.PauseMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.RecoverDatabaseCommand; import org.apache.doris.nereids.trees.plans.commands.RecoverPartitionCommand; import org.apache.doris.nereids.trees.plans.commands.RecoverTableCommand; import org.apache.doris.nereids.trees.plans.commands.RefreshMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.ReplayCommand; +import org.apache.doris.nereids.trees.plans.commands.ResumeJobCommand; import org.apache.doris.nereids.trees.plans.commands.ResumeMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.SetDefaultStorageVaultCommand; import org.apache.doris.nereids.trees.plans.commands.SetOptionsCommand; @@ -164,6 +169,10 @@ default R visitDropConstraintCommand(DropConstraintCommand dropConstraintCommand return visitCommand(dropConstraintCommand, context); } + default R visitDropJobCommand(DropJobCommand dropJobCommand, C context) { + return visitCommand(dropJobCommand, context); + } + default R visitShowConstraintsCommand(ShowConstraintsCommand showConstraintsCommand, C context) { return visitCommand(showConstraintsCommand, context); } @@ -176,10 +185,18 @@ default R visitDropMTMVCommand(DropMTMVCommand dropMTMVCommand, C context) { return visitCommand(dropMTMVCommand, context); } + default R visitPauseJobCommand(PauseJobCommand pauseJobCommand, C context) { + return visitCommand(pauseJobCommand, context); + } + default R visitPauseMTMVCommand(PauseMTMVCommand pauseMTMVCommand, C context) { return visitCommand(pauseMTMVCommand, context); } + default R visitResumeJobCommand(ResumeJobCommand resumeJobCommand, C context) { + return visitCommand(resumeJobCommand, context); + } + default R visitResumeMTMVCommand(ResumeMTMVCommand resumeMTMVCommand, C context) { return visitCommand(resumeMTMVCommand, context); } @@ -192,6 +209,10 @@ default R visitCancelMTMVTaskCommand(CancelMTMVTaskCommand cancelMTMVTaskCommand return visitCommand(cancelMTMVTaskCommand, context); } + default R visitCancelTaskCommand(CancelJobTaskCommand cancelJobTaskCommand, C context) { + return visitCommand(cancelJobTaskCommand, context); + } + default R visitCallCommand(CallCommand callCommand, C context) { return visitCommand(callCommand, context); } @@ -216,6 +237,10 @@ default R visitCreateViewCommand(CreateViewCommand createViewCommand, C context) return visitCommand(createViewCommand, context); } + default R visitAlterJobStatusCommand(AlterJobStatusCommand alterJobStatusCommand, C context) { + return visitCommand(alterJobStatusCommand, context); + } + default R visitAlterViewCommand(AlterViewCommand alterViewCommand, C context) { return visitCommand(alterViewCommand, context); } From 045f59013b0d8da79ee6325923d0cbfa89aa6155 Mon Sep 17 00:00:00 2001 From: echo-dundun <50286010+echo-hhj@users.noreply.github.com> Date: Thu, 21 Nov 2024 15:03:04 +0800 Subject: [PATCH 014/110] [Feat] (Nereids)support showTrash Command (#44352) Issue Number: close #42763 --- .../nereids/parser/LogicalPlanBuilder.java | 7 ++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/ShowTrashCommand.java | 78 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../ddl/show_trash/test_nereids_trash.groovy | 22 ++++++ 5 files changed, 113 insertions(+) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTrashCommand.java create mode 100644 regression-test/suites/nereids_p0/ddl/show_trash/test_nereids_trash.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 00cc011ef23c15a..8b942f2577eb21b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -222,6 +222,7 @@ import org.apache.doris.nereids.DorisParser.ShowStorageEnginesContext; import org.apache.doris.nereids.DorisParser.ShowTableIdContext; import org.apache.doris.nereids.DorisParser.ShowTabletsBelongContext; +import org.apache.doris.nereids.DorisParser.ShowTrashContext; import org.apache.doris.nereids.DorisParser.ShowVariablesContext; import org.apache.doris.nereids.DorisParser.ShowViewContext; import org.apache.doris.nereids.DorisParser.ShowWhitelistContext; @@ -482,6 +483,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowStorageEnginesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTabletsBelongCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowTrashCommand; import org.apache.doris.nereids.trees.plans.commands.ShowVariablesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowWhiteListCommand; @@ -4111,6 +4113,11 @@ public SetDefaultStorageVaultCommand visitSetDefaultStorageVault(SetDefaultStora return new SetDefaultStorageVaultCommand(stripQuotes(ctx.identifier().getText())); } + @Override + public LogicalPlan visitShowTrash(ShowTrashContext ctx) { + return new ShowTrashCommand(); + } + @Override public Object visitRefreshCatalog(RefreshCatalogContext ctx) { if (ctx.name != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index ccfb505624b554c..a29d4d18620474f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -199,6 +199,7 @@ public enum PlanType { SHOW_ROLE_COMMAND, SHOW_STORAGE_ENGINES_COMMAND, SHOW_TABLE_ID_COMMAND, + SHOW_TRASH_COMMAND, SHOW_VARIABLES_COMMAND, SHOW_AUTHORS_COMMAND, SHOW_VIEW_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTrashCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTrashCommand.java new file mode 100644 index 000000000000000..58146cb78575123 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTrashCommand.java @@ -0,0 +1,78 @@ +// 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.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.proc.TrashProcDir; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.system.Backend; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * show trash command + */ +public class ShowTrashCommand extends ShowCommand { + private List backends = Lists.newArrayList(); + + public ShowTrashCommand() { + super(PlanType.SHOW_TRASH_COMMAND); + } + + public List getBackends() { + return backends; + } + + public ShowResultSetMetaData getMetaData() { + ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); + for (String title : TrashProcDir.TITLE_NAMES) { + builder.addColumn(new Column(title, ScalarType.createVarchar(30))); + } + return builder.build(); + } + + private ShowResultSet handleShowTrash() throws Exception { + ImmutableMap backendsInfo = Env.getCurrentSystemInfo().getAllBackendsByAllCluster(); + for (Backend backend : backendsInfo.values()) { + this.backends.add(backend); + } + List> infos = Lists.newArrayList(); + TrashProcDir.getTrashInfo(backends, infos); + return new ShowResultSet(getMetaData(), infos); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowTrashCommand(this, context); + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + return handleShowTrash(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 8e08f75aebbd7ae..960590c4a7af022 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -81,6 +81,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowStorageEnginesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTabletsBelongCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowTrashCommand; import org.apache.doris.nereids.trees.plans.commands.ShowVariablesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowWhiteListCommand; @@ -391,6 +392,10 @@ default R visitShowTableIdCommand(ShowTableIdCommand showTableIdCommand, C conte return visitCommand(showTableIdCommand, context); } + default R visitShowTrashCommand(ShowTrashCommand showTrashCommand, C context) { + return visitCommand(showTrashCommand, context); + } + default R visitShowPrivilegesCommand(ShowPrivilegesCommand showPrivilegesCommand, C context) { return visitCommand(showPrivilegesCommand, context); } diff --git a/regression-test/suites/nereids_p0/ddl/show_trash/test_nereids_trash.groovy b/regression-test/suites/nereids_p0/ddl/show_trash/test_nereids_trash.groovy new file mode 100644 index 000000000000000..5a78b27e41c72e1 --- /dev/null +++ b/regression-test/suites/nereids_p0/ddl/show_trash/test_nereids_trash.groovy @@ -0,0 +1,22 @@ +// 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("show_trash_nereids") { + // can not use qt command since the output change based on cluster and backend ip + checkNereidsExecute("""show trash;""") + checkNereidsExecute("""show trash on "127.0.0.1:9050";""") +} \ No newline at end of file From 0ef2c3750d29f477583c95171e426123a736bfdf Mon Sep 17 00:00:00 2001 From: daidai Date: Thu, 21 Nov 2024 15:35:52 +0800 Subject: [PATCH 015/110] [fix](jvm)fix jvm metrics memory leak. (#44311) ### What problem does this PR solve? Issue Number: close #xxx Related PR: #42507 Problem Summary: fix jvm metrics memory leak. when you set `enable_jvm_monitor=true` in be.conf, you can find that be jvm memory is slowly growing. By analyzing the hprof file, we can find that there are a large number of `java.lang.management.ThreadInfo` objects. The specific cause of the memory leak is: jni does not manually delete the local reference after getting the object from the array, resulting in the object not being GC. --- be/src/util/jvm_metrics.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/be/src/util/jvm_metrics.cpp b/be/src/util/jvm_metrics.cpp index 4cb71f5e827878d..b1089ef413628dc 100644 --- a/be/src/util/jvm_metrics.cpp +++ b/be/src/util/jvm_metrics.cpp @@ -485,8 +485,8 @@ Status JvmStats::refresh(JvmMetrics* jvm_metrics) const { jvm_metrics->jvm_thread_count->set_value(threadCount < 0 ? 0 : threadCount); for (int i = 0; i < threadCount; i++) { - JNI_CALL_METHOD_CHECK_EXCEPTION(jobject, threadInfo, env, - GetObjectArrayElement((jobjectArray)threadInfos, i)); + JNI_CALL_METHOD_CHECK_EXCEPTION_DELETE_REF( + jobject, threadInfo, env, GetObjectArrayElement((jobjectArray)threadInfos, i)); if (threadInfo == nullptr) { continue; From 57957a5d2ed2eb00273fbd92cc6d175a635872df Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Thu, 21 Nov 2024 15:53:28 +0800 Subject: [PATCH 016/110] [Fix](Job)Fix the Calculation of the First Trigger Time and Add a Single-Time Scheduling Compensation Logic (#44268) ### Background: | Time (ms) | Window Start (1000) | First Trigger Time (2000) | Current Time (3000) | Trigger Time (Incorrect Adjustment) | Window End (5000) | |-----------|---------------------|---------------------------|---------------------|-------------------------------------|-------------------| | 0 | | | | | | | 1000 | Start Time | | | | | | 2000 | First Trigger Time | | | | | | 3000 | | Current Time | | | | | 4000 | | | | Trigger Time (incorrect) | | | 5000 | | | | | Window End | The previous scheduling logic for calculating the first trigger time had the following issues: If the current time (currentTimeMs) exceeds the start of the time window (windowStartTimeMs), the first trigger time could be skipped, causing some tasks to be missed. Minor delays during code execution (e.g., between window initialization and scheduling logic execution) could result in tasks being missed within the active time window. ### ### Solution: #### Adjustment of First Trigger Time Logic: When the initially calculated firstTriggerTime is less than or equal to the currentTimeMs, compute the number of missed intervals and directly adjust to the largest trigger time that is less than the currentTimeMs. #### Single-Time Compensation Logic: Introduced a compensation mechanism to handle the slight delay caused by code execution. This ensures that tasks missed during the initialization phase are correctly scheduled. The compensation is explicitly single-time to avoid impacting the normal scheduling logic. --- .../doris/job/base/JobExecutionConfiguration.java | 5 +++-- .../job/base/JobExecutionConfigurationTest.java | 12 ++++++++++-- 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java index 301222d5434ea32..4c6ef4d2037f86c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java @@ -172,9 +172,10 @@ private List getExecutionDelaySeconds(long windowStartTimeMs, long windowE long firstTriggerTime = windowStartTimeMs + (intervalMs - ((windowStartTimeMs - startTimeMs) % intervalMs)) % intervalMs; if (firstTriggerTime < currentTimeMs) { - firstTriggerTime += intervalMs; + // Calculate how many intervals to add to get the largest trigger time < currentTimeMs + long intervalsToAdd = (currentTimeMs - firstTriggerTime) / intervalMs; + firstTriggerTime += intervalsToAdd * intervalMs; } - if (firstTriggerTime > windowEndTimeMs) { return timestamps; // Return an empty list if there won't be any trigger time } diff --git a/fe/fe-core/src/test/java/org/apache/doris/job/base/JobExecutionConfigurationTest.java b/fe/fe-core/src/test/java/org/apache/doris/job/base/JobExecutionConfigurationTest.java index 24c486baff81bcd..cce0a93c01daf86 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/job/base/JobExecutionConfigurationTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/job/base/JobExecutionConfigurationTest.java @@ -60,14 +60,22 @@ public void testGetTriggerDelayTimesRecurring() { Assertions.assertArrayEquals(new Long[]{100L, 700L}, delayTimes.toArray()); delayTimes = configuration.getTriggerDelayTimes( 200000L, 0L, 1100000L); - Assertions.assertEquals(1, delayTimes.size()); - Assertions.assertArrayEquals(new Long[]{500L}, delayTimes.toArray()); + Assertions.assertEquals(2, delayTimes.size()); + Assertions.assertArrayEquals(new Long[]{0L, 500L}, delayTimes.toArray()); delayTimes = configuration.getTriggerDelayTimes( 1001000L, 0L, 1000000L); Assertions.assertEquals(1, delayTimes.size()); timerDefinition.setStartTimeMs(2000L); timerDefinition.setIntervalUnit(IntervalUnit.SECOND); Assertions.assertArrayEquals(new Long[]{2L, 12L}, configuration.getTriggerDelayTimes(100000L, 100000L, 120000L).toArray()); + + timerDefinition.setIntervalUnit(IntervalUnit.SECOND); + long second = 1000L; + timerDefinition.setStartTimeMs(second); + timerDefinition.setInterval(1L); + Assertions.assertEquals(3, configuration.getTriggerDelayTimes(second * 5 + 10L, second * 3, second * 7).size()); + Assertions.assertEquals(3, configuration.getTriggerDelayTimes(second * 5, second * 5, second * 7).size()); + } @Test From 5ed3b9ae25702fa776bf6f7a7f381c444e45a312 Mon Sep 17 00:00:00 2001 From: zy-kkk Date: Thu, 21 Nov 2024 15:55:35 +0800 Subject: [PATCH 017/110] [improvement](jdbc catalog) Optimize JDBC driver property settings (#42923) Abstract JDBC Driver property settings into methods for easier management --- .../main/java/org/apache/doris/jdbc/BaseJdbcExecutor.java | 6 +++++- .../java/org/apache/doris/jdbc/MySQLJdbcExecutor.java | 5 +++++ .../apache/doris/datasource/jdbc/client/JdbcClient.java | 6 +++++- .../doris/datasource/jdbc/client/JdbcMySQLClient.java | 8 ++++++-- 4 files changed, 21 insertions(+), 4 deletions(-) diff --git a/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/BaseJdbcExecutor.java b/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/BaseJdbcExecutor.java index e05a7baa00899f1..4a54ac2d4ce003f 100644 --- a/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/BaseJdbcExecutor.java +++ b/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/BaseJdbcExecutor.java @@ -71,6 +71,7 @@ public abstract class BaseJdbcExecutor implements JdbcExecutor { protected String jdbcDriverVersion; public BaseJdbcExecutor(byte[] thriftParams) throws Exception { + setJdbcDriverSystemProperties(); TJdbcExecutorCtorParams request = new TJdbcExecutorCtorParams(); TDeserializer deserializer = new TDeserializer(PROTOCOL_FACTORY); try { @@ -94,7 +95,6 @@ public BaseJdbcExecutor(byte[] thriftParams) throws Exception { .setConnectionPoolMaxLifeTime(request.connection_pool_max_life_time) .setConnectionPoolKeepAlive(request.connection_pool_keep_alive); JdbcDataSource.getDataSource().setCleanupInterval(request.connection_pool_cache_clear_time); - System.setProperty("com.zaxxer.hikari.useWeakReferences", "true"); init(config, request.statement); this.jdbcDriverVersion = getJdbcDriverVersion(); } @@ -147,6 +147,10 @@ protected void abortReadConnection(Connection connection, ResultSet resultSet) throws SQLException { } + protected void setJdbcDriverSystemProperties() { + System.setProperty("com.zaxxer.hikari.useWeakReferences", "true"); + } + public void cleanDataSource() { if (hikariDataSource != null) { hikariDataSource.close(); diff --git a/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/MySQLJdbcExecutor.java b/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/MySQLJdbcExecutor.java index e94f2b222980721..4e5af95211b5080 100644 --- a/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/MySQLJdbcExecutor.java +++ b/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/MySQLJdbcExecutor.java @@ -50,6 +50,11 @@ public class MySQLJdbcExecutor extends BaseJdbcExecutor { public MySQLJdbcExecutor(byte[] thriftParams) throws Exception { super(thriftParams); + } + + @Override + protected void setJdbcDriverSystemProperties() { + super.setJdbcDriverSystemProperties(); System.setProperty("com.mysql.cj.disableAbandonedConnectionCleanup", "true"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java index 458142ff518fe67..1ab32efddcc1bac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java @@ -100,7 +100,7 @@ public static JdbcClient createJdbcClient(JdbcClientConfig jdbcClientConfig) { } protected JdbcClient(JdbcClientConfig jdbcClientConfig) { - System.setProperty("com.zaxxer.hikari.useWeakReferences", "true"); + setJdbcDriverSystemProperties(); this.catalogName = jdbcClientConfig.getCatalog(); this.jdbcUser = jdbcClientConfig.getUser(); this.isOnlySpecifiedDatabase = Boolean.parseBoolean(jdbcClientConfig.getOnlySpecifiedDatabase()); @@ -117,6 +117,10 @@ protected JdbcClient(JdbcClientConfig jdbcClientConfig) { this.jdbcLowerCaseMetaMatching = new JdbcIdentifierMapping(isLowerCaseMetaNames, metaNamesMapping, this); } + protected void setJdbcDriverSystemProperties() { + System.setProperty("com.zaxxer.hikari.useWeakReferences", "true"); + } + // Initialize DataSource private void initializeDataSource(JdbcClientConfig config) { ClassLoader oldClassLoader = Thread.currentThread().getContextClassLoader(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcMySQLClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcMySQLClient.java index a8263f1621a3a88..b78589faa77380f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcMySQLClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcMySQLClient.java @@ -46,8 +46,6 @@ public class JdbcMySQLClient extends JdbcClient { protected JdbcMySQLClient(JdbcClientConfig jdbcClientConfig) { super(jdbcClientConfig); - // Disable abandoned connection cleanup - System.setProperty("com.mysql.cj.disableAbandonedConnectionCleanup", "true"); convertDateToNull = isConvertDatetimeToNull(jdbcClientConfig); Connection conn = null; Statement stmt = null; @@ -74,6 +72,12 @@ protected JdbcMySQLClient(JdbcClientConfig jdbcClientConfig, String dbType) { this.dbType = dbType; } + @Override + protected void setJdbcDriverSystemProperties() { + super.setJdbcDriverSystemProperties(); + System.setProperty("com.mysql.cj.disableAbandonedConnectionCleanup", "true"); + } + @Override public List getDatabaseNameList() { Connection conn = null; From ede62c61269926f5654f101ec4000a30e6b77143 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Thu, 21 Nov 2024 13:43:47 +0530 Subject: [PATCH 018/110] [Enhancement] (nereids)implement showDynamicPartitionCommand in nereids (#44295) Issue Number: close #42746 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 12 ++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/ShowDynamicPartitionCommand.java | 166 ++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../test_show_dynamic_table_auth.groovy | 6 + 6 files changed, 191 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDynamicPartitionCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index cc7ade25fb2b5fc..2ab16278525a94a 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -203,6 +203,7 @@ supportedShowStatement : SHOW (GLOBAL | SESSION | LOCAL)? VARIABLES wildWhere? #showVariables | SHOW AUTHORS #showAuthors + | SHOW DYNAMIC PARTITION TABLES ((FROM | IN) database=multipartIdentifier)? #showDynamicPartition | SHOW LAST INSERT #showLastInsert | SHOW ALL? GRANTS #showGrants | SHOW GRANTS FOR userIdentify #showGrantsForUser @@ -281,7 +282,6 @@ unsupportedShowStatement | SHOW DATA TYPES #showDataTypes | SHOW CATALOGS wildWhere? #showCatalogs | SHOW CATALOG name=identifier #showCatalog - | SHOW DYNAMIC PARTITION TABLES ((FROM | IN) database=multipartIdentifier)? #showDynamicPartition | SHOW FULL? (COLUMNS | FIELDS) (FROM | IN) tableName=multipartIdentifier ((FROM | IN) database=multipartIdentifier)? wildWhere? #showColumns | SHOW COLLATION wildWhere? #showCollation diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 8b942f2577eb21b..fd84a448c7c0fc2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -207,6 +207,7 @@ import org.apache.doris.nereids.DorisParser.ShowCreateMaterializedViewContext; import org.apache.doris.nereids.DorisParser.ShowCreateProcedureContext; import org.apache.doris.nereids.DorisParser.ShowCreateTableContext; +import org.apache.doris.nereids.DorisParser.ShowDynamicPartitionContext; import org.apache.doris.nereids.DorisParser.ShowFrontendsContext; import org.apache.doris.nereids.DorisParser.ShowGrantsContext; import org.apache.doris.nereids.DorisParser.ShowGrantsForUserContext; @@ -469,6 +470,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowCreateMaterializedViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateProcedureCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateTableCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowDynamicPartitionCommand; import org.apache.doris.nereids.trees.plans.commands.ShowFrontendsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowGrantsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowLastInsertCommand; @@ -4279,6 +4281,16 @@ public LogicalPlan visitShowWhitelist(ShowWhitelistContext ctx) { return new ShowWhiteListCommand(); } + @Override + public LogicalPlan visitShowDynamicPartition(ShowDynamicPartitionContext ctx) { + String dbName = null; + if (ctx.database != null) { + List nameParts = visitMultipartIdentifier(ctx.database); + dbName = nameParts.get(0); // only one entry possible + } + return new ShowDynamicPartitionCommand(dbName); + } + @Override public LogicalPlan visitRecoverDatabase(RecoverDatabaseContext ctx) { String dbName = ctx.name.getText(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index a29d4d18620474f..291bfd3697ad6af 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -188,6 +188,7 @@ public enum PlanType { SHOW_CREATE_CATALOG_COMMAND, SHOW_CREATE_MATERIALIZED_VIEW_COMMAND, SHOW_CREATE_TABLE_COMMAND, + SHOW_DYNAMIC_PARTITION_COMMAND, SHOW_FRONTENDS_COMMAND, SHOW_GRANTS_COMMAND, SHOW_LAST_INSERT_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDynamicPartitionCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDynamicPartitionCommand.java new file mode 100644 index 000000000000000..12063467a323f3a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDynamicPartitionCommand.java @@ -0,0 +1,166 @@ +// 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.nereids.trees.plans.commands; + +import org.apache.doris.analysis.RedirectStatus; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.DynamicPartitionProperty; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.ReplicaAllocation; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.catalog.Table; +import org.apache.doris.clone.DynamicPartitionScheduler; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; + +/** + * show dynamic partition command + */ +public class ShowDynamicPartitionCommand extends ShowCommand { + private static final Logger LOG = LogManager.getLogger(ShowDynamicPartitionCommand.class); + private static final ShowResultSetMetaData SHOW_DYNAMIC_PARTITION_META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("TableName", ScalarType.createVarchar(20))) + .addColumn(new Column("Enable", ScalarType.createVarchar(20))) + .addColumn(new Column("TimeUnit", ScalarType.createVarchar(20))) + .addColumn(new Column("Start", ScalarType.createVarchar(20))) + .addColumn(new Column("End", ScalarType.createVarchar(20))) + .addColumn(new Column("Prefix", ScalarType.createVarchar(20))) + .addColumn(new Column("Buckets", ScalarType.createVarchar(20))) + .addColumn(new Column("ReplicationNum", ScalarType.createVarchar(20))) + .addColumn(new Column("ReplicaAllocation", ScalarType.createVarchar(128))) + .addColumn(new Column("StartOf", ScalarType.createVarchar(20))) + .addColumn(new Column("LastUpdateTime", ScalarType.createVarchar(20))) + .addColumn(new Column("LastSchedulerTime", ScalarType.createVarchar(20))) + .addColumn(new Column("State", ScalarType.createVarchar(20))) + .addColumn(new Column("LastCreatePartitionMsg", ScalarType.createVarchar(20))) + .addColumn(new Column("LastDropPartitionMsg", ScalarType.createVarchar(20))) + .addColumn(new Column("ReservedHistoryPeriods", ScalarType.createVarchar(20))) + .build(); + private String dbName; // if empty we will use current db; + + /** + * constructor + */ + public ShowDynamicPartitionCommand(String dbName) { + super(PlanType.SHOW_DYNAMIC_PARTITION_COMMAND); + this.dbName = dbName; + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + if (Strings.isNullOrEmpty(dbName)) { + dbName = ctx.getDatabase(); + if (Strings.isNullOrEmpty(dbName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); + } + } + List> rows = Lists.newArrayList(); + DatabaseIf db = ctx.getEnv().getInternalCatalog().getDbOrAnalysisException(dbName); + if (db != null && (db instanceof Database)) { + List tableList = db.getTables(); + for (Table tbl : tableList) { + if (!(tbl instanceof OlapTable)) { + continue; + } + DynamicPartitionScheduler dynamicPartitionScheduler = Env.getCurrentEnv() + .getDynamicPartitionScheduler(); + OlapTable olapTable = (OlapTable) tbl; + olapTable.readLock(); + try { + if (!olapTable.dynamicPartitionExists()) { + dynamicPartitionScheduler.removeRuntimeInfo(olapTable.getId()); + continue; + } + // check tbl privs + if (!Env.getCurrentEnv().getAccessManager() + .checkTblPriv(ConnectContext.get(), InternalCatalog.INTERNAL_CATALOG_NAME, db.getFullName(), + olapTable.getName(), + PrivPredicate.SHOW)) { + continue; + } + DynamicPartitionProperty dynamicPartitionProperty + = olapTable.getTableProperty().getDynamicPartitionProperty(); + String tableName = olapTable.getName(); + ReplicaAllocation replicaAlloc = dynamicPartitionProperty.getReplicaAllocation(); + if (replicaAlloc.isNotSet()) { + replicaAlloc = olapTable.getDefaultReplicaAllocation(); + } + String unsortedReservedHistoryPeriods = dynamicPartitionProperty.getReservedHistoryPeriods(); + rows.add(Lists.newArrayList( + tableName, + String.valueOf(dynamicPartitionProperty.getEnable()), + dynamicPartitionProperty.getTimeUnit().toUpperCase(), + String.valueOf(dynamicPartitionProperty.getStart()), + String.valueOf(dynamicPartitionProperty.getEnd()), + dynamicPartitionProperty.getPrefix(), + String.valueOf(dynamicPartitionProperty.getBuckets()), + String.valueOf(replicaAlloc.getTotalReplicaNum()), + replicaAlloc.toCreateStmt(), + dynamicPartitionProperty.getStartOfInfo(), + dynamicPartitionScheduler.getRuntimeInfo(olapTable.getId(), + DynamicPartitionScheduler.LAST_UPDATE_TIME), + dynamicPartitionScheduler.getRuntimeInfo(olapTable.getId(), + DynamicPartitionScheduler.LAST_SCHEDULER_TIME), + dynamicPartitionScheduler.getRuntimeInfo(olapTable.getId(), + DynamicPartitionScheduler.DYNAMIC_PARTITION_STATE), + dynamicPartitionScheduler.getRuntimeInfo(olapTable.getId(), + DynamicPartitionScheduler.CREATE_PARTITION_MSG), + dynamicPartitionScheduler.getRuntimeInfo(olapTable.getId(), + DynamicPartitionScheduler.DROP_PARTITION_MSG), + dynamicPartitionProperty.getSortedReservedHistoryPeriods(unsortedReservedHistoryPeriods, + dynamicPartitionProperty.getTimeUnit().toUpperCase()))); + } catch (DdlException e) { + LOG.warn("", e); + } finally { + olapTable.readUnlock(); + } + } + } + return new ShowResultSet(SHOW_DYNAMIC_PARTITION_META_DATA, rows); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowDynamicPartitionCommand(this, context); + } + + @Override + public RedirectStatus toRedirectStatus() { + return RedirectStatus.FORWARD_NO_SYNC; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 960590c4a7af022..50df22f209fcd25 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -67,6 +67,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowCreateMaterializedViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateProcedureCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateTableCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowDynamicPartitionCommand; import org.apache.doris.nereids.trees.plans.commands.ShowFrontendsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowGrantsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowLastInsertCommand; @@ -364,6 +365,10 @@ default R visitShowFrontendsCommand(ShowFrontendsCommand showFrontendsCommand, C return visitCommand(showFrontendsCommand, context); } + default R visitShowDynamicPartitionCommand(ShowDynamicPartitionCommand showDynamicPartitionCommand, C context) { + return visitCommand(showDynamicPartitionCommand, context); + } + default R visitShowWhiteListCommand(ShowWhiteListCommand whiteListCommand, C context) { return visitCommand(whiteListCommand, context); } diff --git a/regression-test/suites/auth_call/test_show_dynamic_table_auth.groovy b/regression-test/suites/auth_call/test_show_dynamic_table_auth.groovy index e144d87749c193c..248e1d1c3559d74 100644 --- a/regression-test/suites/auth_call/test_show_dynamic_table_auth.groovy +++ b/regression-test/suites/auth_call/test_show_dynamic_table_auth.groovy @@ -58,8 +58,14 @@ suite("test_show_dynamic_table_auth","p0,auth_call") { } sql """grant select_priv on ${dbName}.${tableName} to ${user}""" connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + + checkNereidsExecute("SHOW DYNAMIC PARTITION TABLES from ${dbName};"); + def res = sql """SHOW DYNAMIC PARTITION TABLES from ${dbName};""" assertTrue(res.size() == 1) + + sql """ use ${dbName};""" + checkNereidsExecute("SHOW DYNAMIC PARTITION TABLES;"); } sql """drop database if exists ${dbName}""" From f663443fcbe73aa27aff8ff64a4e065ba505bde5 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Thu, 21 Nov 2024 13:44:38 +0530 Subject: [PATCH 019/110] [Enhancement] (nereids)implement CleanAllProfileCommand in nereids (#44318) Issue Number: close #42570 --- .../org/apache/doris/nereids/DorisParser.g4 | 6 +- .../nereids/parser/LogicalPlanBuilder.java | 7 ++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/CleanAllProfileCommand.java | 66 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../test_dml_cancel_profile_auth.groovy | 1 + 6 files changed, 85 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CleanAllProfileCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 2ab16278525a94a..177eff9b6db5432 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -52,6 +52,7 @@ statementBase | materializedViewStatement #materializedViewStatementAlias | supportedJobStatement #supportedJobStatementAlias | constraintStatement #constraintStatementAlias + | supportedCleanStatement #supportedCleanStatementAlias | supportedDropStatement #supportedDropStatementAlias | supportedSetStatement #supportedSetStatementAlias | supportedUnsetStatement #supportedUnsetStatementAlias @@ -426,6 +427,10 @@ supportedRefreshStatement : REFRESH CATALOG name=identifier propertyClause? #refreshCatalog ; +supportedCleanStatement + : CLEAN ALL PROFILE #cleanAllProfile + ; + unsupportedRefreshStatement : REFRESH TABLE name=multipartIdentifier #refreshTable | REFRESH DATABASE name=multipartIdentifier propertyClause? #refreshDatabase @@ -434,7 +439,6 @@ unsupportedRefreshStatement unsupportedCleanStatement : CLEAN LABEL label=identifier? (FROM | IN) database=identifier #cleanLabel - | CLEAN ALL PROFILE #cleanAllProfile | CLEAN QUERY STATS ((FOR database=identifier) | ((FROM | IN) table=multipartIdentifier)) #cleanQueryStats | CLEAN ALL QUERY STATS #cleanAllQueryStats diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index fd84a448c7c0fc2..d24e4c1fa7bb99b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -70,6 +70,7 @@ import org.apache.doris.nereids.DorisParser.CallProcedureContext; import org.apache.doris.nereids.DorisParser.CancelMTMVTaskContext; import org.apache.doris.nereids.DorisParser.CastDataTypeContext; +import org.apache.doris.nereids.DorisParser.CleanAllProfileContext; import org.apache.doris.nereids.DorisParser.CollateContext; import org.apache.doris.nereids.DorisParser.ColumnDefContext; import org.apache.doris.nereids.DorisParser.ColumnDefsContext; @@ -425,6 +426,7 @@ import org.apache.doris.nereids.trees.plans.commands.CallCommand; import org.apache.doris.nereids.trees.plans.commands.CancelJobTaskCommand; import org.apache.doris.nereids.trees.plans.commands.CancelMTMVTaskCommand; +import org.apache.doris.nereids.trees.plans.commands.CleanAllProfileCommand; import org.apache.doris.nereids.trees.plans.commands.Command; import org.apache.doris.nereids.trees.plans.commands.Constraint; import org.apache.doris.nereids.trees.plans.commands.CreateJobCommand; @@ -4276,6 +4278,11 @@ public LogicalPlan visitShowFrontends(ShowFrontendsContext ctx) { return new ShowFrontendsCommand(detail); } + @Override + public LogicalPlan visitCleanAllProfile(CleanAllProfileContext ctx) { + return new CleanAllProfileCommand(); + } + @Override public LogicalPlan visitShowWhitelist(ShowWhitelistContext ctx) { return new ShowWhiteListCommand(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 291bfd3697ad6af..c3d7fc70801447e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -166,6 +166,7 @@ public enum PlanType { SHOW_PROCEDURE_COMMAND, SHOW_CREATE_PROCEDURE_COMMAND, CREATE_VIEW_COMMAND, + CLEAN_ALL_PROFILE_COMMAND, ALTER_ROLE_COMMAND, ALTER_VIEW_COMMAND, ALTER_STORAGE_VAULT, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CleanAllProfileCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CleanAllProfileCommand.java new file mode 100644 index 000000000000000..937148b5989e1b3 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CleanAllProfileCommand.java @@ -0,0 +1,66 @@ +// 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.nereids.trees.plans.commands; + +import org.apache.doris.analysis.RedirectStatus; +import org.apache.doris.analysis.StmtType; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.profile.ProfileManager; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +/** + * clean profile command + */ +public class CleanAllProfileCommand extends Command implements Redirect { + + /** + * constructor + */ + public CleanAllProfileCommand() { + super(PlanType.CLEAN_ALL_PROFILE_COMMAND); + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + ProfileManager.getInstance().cleanProfile(); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCleanAllProfileCommand(this, context); + } + + @Override + public RedirectStatus toRedirectStatus() { + return RedirectStatus.NO_FORWARD; + } + + @Override + public StmtType stmtType() { + return StmtType.CLEAN; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 50df22f209fcd25..7aa935df8bfe62d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -25,6 +25,7 @@ import org.apache.doris.nereids.trees.plans.commands.CallCommand; import org.apache.doris.nereids.trees.plans.commands.CancelJobTaskCommand; import org.apache.doris.nereids.trees.plans.commands.CancelMTMVTaskCommand; +import org.apache.doris.nereids.trees.plans.commands.CleanAllProfileCommand; import org.apache.doris.nereids.trees.plans.commands.Command; import org.apache.doris.nereids.trees.plans.commands.CreateJobCommand; import org.apache.doris.nereids.trees.plans.commands.CreateMTMVCommand; @@ -361,6 +362,10 @@ default R visitAlterRoleCommand(AlterRoleCommand alterRoleCommand, C context) { return visitCommand(alterRoleCommand, context); } + default R visitCleanAllProfileCommand(CleanAllProfileCommand cleanAllProfileCommand, C context) { + return visitCommand(cleanAllProfileCommand, context); + } + default R visitShowFrontendsCommand(ShowFrontendsCommand showFrontendsCommand, C context) { return visitCommand(showFrontendsCommand, context); } diff --git a/regression-test/suites/auth_call/test_dml_cancel_profile_auth.groovy b/regression-test/suites/auth_call/test_dml_cancel_profile_auth.groovy index 82656726e659cee..84c5a5ef5ba20e5 100644 --- a/regression-test/suites/auth_call/test_dml_cancel_profile_auth.groovy +++ b/regression-test/suites/auth_call/test_dml_cancel_profile_auth.groovy @@ -45,6 +45,7 @@ suite("test_dml_cancel_profile_auth","p0,auth_call,nonConcurrent") { } sql """grant admin_priv on *.*.* to ${user}""" connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + checkNereidsExecute("CLEAN ALL PROFILE") sql """ CLEAN ALL PROFILE; """ From 3352d84a8e4180cbc88881cca5e3fa73313a8d91 Mon Sep 17 00:00:00 2001 From: walter Date: Thu, 21 Nov 2024 17:44:12 +0800 Subject: [PATCH 020/110] [feat](binlog) Add replace table binlog (#44263) close #43436 Related PR: https://github.com/selectdb/ccr-syncer/pull/245 --- .../java/org/apache/doris/alter/Alter.java | 2 +- .../apache/doris/binlog/BinlogManager.java | 18 +++++++ .../org/apache/doris/binlog/DBBinlog.java | 53 ++++++++++++++----- .../org/apache/doris/persist/BarrierLog.java | 4 ++ .../org/apache/doris/persist/EditLog.java | 11 ++-- .../persist/ReplaceTableOperationLog.java | 28 +++++++++- .../persist/ReplaceTableOperationLogTest.java | 4 +- gensrc/thrift/FrontendService.thrift | 4 +- 8 files changed, 101 insertions(+), 23 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index 15c8df9195ddf16..ebb194ed6a6262b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -603,7 +603,7 @@ public void processReplaceTable(Database db, OlapTable origTable, String newTblN replaceTableInternal(db, origTable, olapNewTbl, swapTable, false, isForce); // write edit log ReplaceTableOperationLog log = new ReplaceTableOperationLog(db.getId(), - origTable.getId(), olapNewTbl.getId(), swapTable, isForce); + origTable.getId(), oldTblName, olapNewTbl.getId(), newTblName, swapTable, isForce); Env.getCurrentEnv().getEditLog().logReplaceTable(log); LOG.info("finish replacing table {} with table {}, is swap: {}", oldTblName, newTblName, swapTable); } finally { diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java index 6d483a413141112..1f785713666437f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java @@ -33,6 +33,7 @@ import org.apache.doris.persist.ModifyCommentOperationLog; import org.apache.doris.persist.ModifyTablePropertyOperationLog; import org.apache.doris.persist.ReplacePartitionOperationLog; +import org.apache.doris.persist.ReplaceTableOperationLog; import org.apache.doris.persist.TableAddOrDropColumnsInfo; import org.apache.doris.persist.TableInfo; import org.apache.doris.persist.TableRenameColumnInfo; @@ -45,6 +46,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.thrift.TException; @@ -367,6 +369,22 @@ public void addModifyViewDef(AlterViewInfo alterViewInfo, long commitSeq) { addBinlog(dbId, tableIds, commitSeq, timestamp, type, data, false, alterViewInfo); } + public void addReplaceTable(ReplaceTableOperationLog info, long commitSeq) { + if (StringUtils.isEmpty(info.getOrigTblName()) || StringUtils.isEmpty(info.getNewTblName())) { + LOG.warn("skip replace table binlog, because origTblName or newTblName is empty. info: {}", info); + return; + } + + long dbId = info.getDbId(); + List tableIds = Lists.newArrayList(); + tableIds.add(info.getOrigTblId()); + long timestamp = -1; + TBinlogType type = TBinlogType.REPLACE_TABLE; + String data = info.toJson(); + + addBinlog(dbId, tableIds, commitSeq, timestamp, type, data, false, info); + } + // get binlog by dbId, return first binlog.version > version public Pair getBinlog(long dbId, long tableId, long prevCommitSeq) { TStatus status = new TStatus(TStatusCode.OK); diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/DBBinlog.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/DBBinlog.java index e2eef7966be0d4c..c96e994be91c3ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/DBBinlog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/DBBinlog.java @@ -22,7 +22,9 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.Pair; import org.apache.doris.common.proc.BaseProcResult; +import org.apache.doris.persist.BarrierLog; import org.apache.doris.persist.DropPartitionInfo; +import org.apache.doris.persist.ReplaceTableOperationLog; import org.apache.doris.thrift.TBinlog; import org.apache.doris.thrift.TBinlogType; import org.apache.doris.thrift.TStatus; @@ -626,19 +628,29 @@ private void recordDroppedResources(TBinlog binlog) { // A method to record the dropped tables, indexes, and partitions. private void recordDroppedResources(TBinlog binlog, Object raw) { + recordDroppedResources(binlog.getType(), binlog.getCommitSeq(), binlog.getData(), raw); + } + + private void recordDroppedResources(TBinlogType binlogType, long commitSeq, String data, Object raw) { if (raw == null) { - switch (binlog.getType()) { + switch (binlogType) { case DROP_PARTITION: - raw = DropPartitionInfo.fromJson(binlog.data); + raw = DropPartitionInfo.fromJson(data); break; case DROP_TABLE: - raw = DropTableRecord.fromJson(binlog.data); + raw = DropTableRecord.fromJson(data); break; case ALTER_JOB: - raw = AlterJobRecord.fromJson(binlog.data); + raw = AlterJobRecord.fromJson(data); break; case TRUNCATE_TABLE: - raw = TruncateTableRecord.fromJson(binlog.data); + raw = TruncateTableRecord.fromJson(data); + break; + case REPLACE_TABLE: + raw = ReplaceTableOperationLog.fromJson(data); + break; + case BARRIER: + raw = BarrierLog.fromJson(data); break; default: break; @@ -648,29 +660,44 @@ private void recordDroppedResources(TBinlog binlog, Object raw) { } } - if (binlog.getType() == TBinlogType.DROP_PARTITION && raw instanceof DropPartitionInfo) { + recordDroppedResources(binlogType, commitSeq, raw); + } + + private void recordDroppedResources(TBinlogType binlogType, long commitSeq, Object raw) { + if (binlogType == TBinlogType.DROP_PARTITION && raw instanceof DropPartitionInfo) { long partitionId = ((DropPartitionInfo) raw).getPartitionId(); if (partitionId > 0) { - droppedPartitions.add(Pair.of(partitionId, binlog.getCommitSeq())); + droppedPartitions.add(Pair.of(partitionId, commitSeq)); } - } else if (binlog.getType() == TBinlogType.DROP_TABLE && raw instanceof DropTableRecord) { + } else if (binlogType == TBinlogType.DROP_TABLE && raw instanceof DropTableRecord) { long tableId = ((DropTableRecord) raw).getTableId(); if (tableId > 0) { - droppedTables.add(Pair.of(tableId, binlog.getCommitSeq())); + droppedTables.add(Pair.of(tableId, commitSeq)); } - } else if (binlog.getType() == TBinlogType.ALTER_JOB && raw instanceof AlterJobRecord) { + } else if (binlogType == TBinlogType.ALTER_JOB && raw instanceof AlterJobRecord) { AlterJobRecord alterJobRecord = (AlterJobRecord) raw; if (alterJobRecord.isJobFinished() && alterJobRecord.isSchemaChangeJob()) { for (Long indexId : alterJobRecord.getOriginIndexIdList()) { if (indexId != null && indexId > 0) { - droppedIndexes.add(Pair.of(indexId, binlog.getCommitSeq())); + droppedIndexes.add(Pair.of(indexId, commitSeq)); } } } - } else if (binlog.getType() == TBinlogType.TRUNCATE_TABLE && raw instanceof TruncateTableRecord) { + } else if (binlogType == TBinlogType.TRUNCATE_TABLE && raw instanceof TruncateTableRecord) { TruncateTableRecord truncateTableRecord = (TruncateTableRecord) raw; for (long partitionId : truncateTableRecord.getOldPartitionIds()) { - droppedPartitions.add(Pair.of(partitionId, binlog.getCommitSeq())); + droppedPartitions.add(Pair.of(partitionId, commitSeq)); + } + } else if (binlogType == TBinlogType.REPLACE_TABLE && raw instanceof ReplaceTableOperationLog) { + ReplaceTableOperationLog record = (ReplaceTableOperationLog) raw; + if (!record.isSwapTable()) { + droppedTables.add(Pair.of(record.getOrigTblId(), commitSeq)); + } + } else if (binlogType == TBinlogType.BARRIER && raw instanceof BarrierLog) { + BarrierLog log = (BarrierLog) raw; + // keep compatible with doris 2.0/2.1 + if (log.hasBinlog()) { + recordDroppedResources(log.getBinlogType(), commitSeq, log.getBinlog(), null); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/BarrierLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/BarrierLog.java index 4a9ce13e03b3ed2..86d56fb4a6487e6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/BarrierLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/BarrierLog.java @@ -109,6 +109,10 @@ public String toJson() { return GsonUtils.GSON.toJson(this); } + public static BarrierLog fromJson(String json) { + return GsonUtils.GSON.fromJson(json, BarrierLog.class); + } + @Override public String toString() { return toJson(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index df0cdb092a8b539..5ae6f62ebb20e93 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -306,7 +306,7 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { case OperationType.OP_RENAME_TABLE: { TableInfo info = (TableInfo) journal.getData(); env.replayRenameTable(info); - Env.getCurrentEnv().getBinlogManager().addTableRename(info, logId); + env.getBinlogManager().addTableRename(info, logId); break; } case OperationType.OP_MODIFY_VIEW_DEF: { @@ -318,7 +318,7 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { case OperationType.OP_RENAME_PARTITION: { TableInfo info = (TableInfo) journal.getData(); env.replayRenamePartition(info); - Env.getCurrentEnv().getBinlogManager().addTableRename(info, logId); + env.getBinlogManager().addTableRename(info, logId); break; } case OperationType.OP_RENAME_COLUMN: { @@ -366,7 +366,7 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { case OperationType.OP_RENAME_ROLLUP: { TableInfo info = (TableInfo) journal.getData(); env.replayRenameRollup(info); - Env.getCurrentEnv().getBinlogManager().addTableRename(info, logId); + env.getCurrentEnv().getBinlogManager().addTableRename(info, logId); break; } case OperationType.OP_LOAD_START: @@ -898,6 +898,7 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { case OperationType.OP_REPLACE_TABLE: { ReplaceTableOperationLog log = (ReplaceTableOperationLog) journal.getData(); env.getAlterInstance().replayReplaceTable(log); + env.getBinlogManager().addReplaceTable(log, logId); break; } case OperationType.OP_CREATE_SQL_BLOCK_RULE: { @@ -1950,7 +1951,9 @@ public void logGlobalVariableV2(GlobalVarPersistInfo info) { } public void logReplaceTable(ReplaceTableOperationLog log) { - logEdit(OperationType.OP_REPLACE_TABLE, log); + long logId = logEdit(OperationType.OP_REPLACE_TABLE, log); + LOG.info("add replace table binlog, logId: {}, infos: {}", logId, log); + Env.getCurrentEnv().getBinlogManager().addReplaceTable(log, logId); } public void logBatchRemoveTransactions(BatchRemoveTransactionsOperationV2 op) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/ReplaceTableOperationLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/ReplaceTableOperationLog.java index 7a685f3741f51db..6a2b09336e1eaed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/ReplaceTableOperationLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/ReplaceTableOperationLog.java @@ -32,17 +32,25 @@ public class ReplaceTableOperationLog implements Writable { private long dbId; @SerializedName(value = "origTblId") private long origTblId; + @SerializedName(value = "origTblName") + private String origTblName; @SerializedName(value = "newTblName") private long newTblId; + @SerializedName(value = "actualNewTblName") + private String newTblName; @SerializedName(value = "swapTable") private boolean swapTable; @SerializedName(value = "isForce") private boolean isForce = true; // older version it was force. so keep same. - public ReplaceTableOperationLog(long dbId, long origTblId, long newTblId, boolean swapTable, boolean isForce) { + public ReplaceTableOperationLog(long dbId, long origTblId, + String origTblName, long newTblId, String newTblName, + boolean swapTable, boolean isForce) { this.dbId = dbId; this.origTblId = origTblId; + this.origTblName = origTblName; this.newTblId = newTblId; + this.newTblName = newTblName; this.swapTable = swapTable; this.isForce = isForce; } @@ -55,10 +63,18 @@ public long getOrigTblId() { return origTblId; } + public String getOrigTblName() { + return origTblName; + } + public long getNewTblId() { return newTblId; } + public String getNewTblName() { + return newTblName; + } + public boolean isSwapTable() { return swapTable; } @@ -67,13 +83,21 @@ public boolean isForce() { return isForce; } + public String toJson() { + return GsonUtils.GSON.toJson(this); + } + + public static ReplaceTableOperationLog fromJson(String json) { + return GsonUtils.GSON.fromJson(json, ReplaceTableOperationLog.class); + } + @Override public void write(DataOutput out) throws IOException { String json = GsonUtils.GSON.toJson(this); Text.writeString(out, json); } - public static ReplaceTableOperationLog read(DataInput in) throws IOException { + public static ReplaceTableOperationLog read(DataInput in) throws IOException { String json = Text.readString(in); return GsonUtils.GSON.fromJson(json, ReplaceTableOperationLog.class); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/persist/ReplaceTableOperationLogTest.java b/fe/fe-core/src/test/java/org/apache/doris/persist/ReplaceTableOperationLogTest.java index e05d16141ced721..ed56e4c7941342b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/persist/ReplaceTableOperationLogTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/persist/ReplaceTableOperationLogTest.java @@ -34,7 +34,7 @@ public void testSerialization() throws Exception { file.createNewFile(); DataOutputStream dos = new DataOutputStream(new FileOutputStream(file)); - ReplaceTableOperationLog log = new ReplaceTableOperationLog(1, 2, 3, true, true); + ReplaceTableOperationLog log = new ReplaceTableOperationLog(1, 2, "old", 3, "new", true, true); log.write(dos); dos.flush(); @@ -48,6 +48,8 @@ public void testSerialization() throws Exception { Assert.assertTrue(readLog.getNewTblId() == log.getNewTblId()); Assert.assertTrue(readLog.getOrigTblId() == log.getOrigTblId()); Assert.assertTrue(readLog.isSwapTable() == log.isSwapTable()); + Assert.assertTrue(readLog.getOrigTblName().equals(log.getOrigTblName())); + Assert.assertTrue(readLog.getNewTblName().equals(log.getNewTblName())); // 3. delete files dis.close(); diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index ec2a685098ce699..47b88552862254e 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -1191,6 +1191,7 @@ enum TBinlogType { RENAME_COLUMN = 15, MODIFY_COMMENT = 16, MODIFY_VIEW_DEF = 17, + REPLACE_TABLE = 18, // Keep some IDs for allocation so that when new binlog types are added in the // future, the changes can be picked back to the old versions without breaking @@ -1207,8 +1208,7 @@ enum TBinlogType { // MODIFY_XXX = 17, // MIN_UNKNOWN = 18, // UNKNOWN_3 = 19, - MIN_UNKNOWN = 18, - UNKNOWN_3 = 19, + MIN_UNKNOWN = 19, UNKNOWN_4 = 20, UNKNOWN_5 = 21, UNKNOWN_6 = 22, From 29530553244e4375c40124612071a6203e2c249c Mon Sep 17 00:00:00 2001 From: zzzxl Date: Thu, 21 Nov 2024 18:55:42 +0800 Subject: [PATCH 021/110] [fix](inverted index) fix index-tool compilation errors (#44320) Problem Summary: fix index-tool compilation errors --- be/src/index-tools/index_tool.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/be/src/index-tools/index_tool.cpp b/be/src/index-tools/index_tool.cpp index ca0575dc545459d..e45902c0f24df15 100644 --- a/be/src/index-tools/index_tool.cpp +++ b/be/src/index-tools/index_tool.cpp @@ -562,7 +562,7 @@ int main(int argc, char** argv) { auto dir = std::forward(st).value(); auto analyzer = _CLNEW lucene::analysis::standard95::StandardAnalyzer(); // auto analyzer = _CLNEW lucene::analysis::SimpleAnalyzer(); - auto indexwriter = _CLNEW lucene::index::IndexWriter(dir, analyzer, true, true); + auto indexwriter = _CLNEW lucene::index::IndexWriter(dir.get(), analyzer, true, true); indexwriter->setRAMBufferSizeMB(512); indexwriter->setMaxFieldLength(0x7FFFFFFFL); indexwriter->setMergeFactor(100000000); From bdef6013a8c8b263443ebe6c00d10a9c6c41da8b Mon Sep 17 00:00:00 2001 From: zhiqiang Date: Thu, 21 Nov 2024 20:47:05 +0800 Subject: [PATCH 022/110] [chore](type cast) Fix some implicit cast (#43050) --- .../vec/data_types/convert_field_to_type.cpp | 5 +- be/src/vec/data_types/data_type.cpp | 4 +- be/src/vec/data_types/data_type.h | 12 +++- be/src/vec/data_types/data_type_bitmap.cpp | 4 +- be/src/vec/data_types/data_type_date.h | 3 +- be/src/vec/data_types/data_type_date_time.h | 2 + be/src/vec/data_types/data_type_decimal.cpp | 26 ++++++--- be/src/vec/data_types/data_type_factory.cpp | 14 ++--- .../data_type_fixed_length_object.cpp | 5 +- be/src/vec/data_types/data_type_jsonb.cpp | 7 ++- be/src/vec/data_types/data_type_jsonb.h | 9 ++- be/src/vec/data_types/data_type_nullable.cpp | 26 ++++++--- .../vec/data_types/data_type_number_base.cpp | 33 ++++++----- be/src/vec/data_types/data_type_number_base.h | 8 ++- be/src/vec/data_types/data_type_object.cpp | 15 +++-- .../data_types/data_type_quantilestate.cpp | 4 +- be/src/vec/data_types/data_type_string.cpp | 58 ++++++++++++------- be/src/vec/data_types/data_type_time_v2.cpp | 1 + be/src/vec/data_types/data_type_time_v2.h | 12 ++-- be/src/vec/runtime/vdatetime_value.cpp | 9 +-- be/src/vec/runtime/vdatetime_value.h | 24 ++++---- 21 files changed, 178 insertions(+), 103 deletions(-) diff --git a/be/src/vec/data_types/convert_field_to_type.cpp b/be/src/vec/data_types/convert_field_to_type.cpp index ecbce03ba6b10a9..c625f8c424bba6d 100644 --- a/be/src/vec/data_types/convert_field_to_type.cpp +++ b/be/src/vec/data_types/convert_field_to_type.cpp @@ -30,6 +30,7 @@ #include #include +#include "common/cast_set.h" #include "common/exception.h" #include "common/status.h" #include "util/bitmap_value.h" @@ -44,6 +45,7 @@ #include "vec/data_types/data_type_nullable.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" /** Checking for a `Field from` of `From` type falls to a range of values of type `To`. * `From` and `To` - numeric types. They can be floating-point types. * `From` is one of UInt64, Int64, Float64, @@ -257,7 +259,8 @@ void convert_field_to_typeImpl(const Field& src, const IDataType& type, JsonbWriter writer; Field::dispatch([&writer](const auto& value) { FieldVisitorToJsonb()(value, &writer); }, src); - *to = JsonbField(writer.getOutput()->getBuffer(), writer.getOutput()->getSize()); + *to = JsonbField(writer.getOutput()->getBuffer(), + cast_set(writer.getOutput()->getSize())); return; } else if (which_type.is_variant_type()) { if (src.get_type() == Field::Types::VariantMap) { diff --git a/be/src/vec/data_types/data_type.cpp b/be/src/vec/data_types/data_type.cpp index cacbf4b2ecec3f5..28415076ba339a3 100644 --- a/be/src/vec/data_types/data_type.cpp +++ b/be/src/vec/data_types/data_type.cpp @@ -41,6 +41,7 @@ class ReadBuffer; } // namespace doris namespace doris::vectorized { +#include "common/compile_check_begin.h" IDataType::IDataType() = default; @@ -58,7 +59,8 @@ void IDataType::update_avg_value_size_hint(const IColumn& column, double& avg_va /// Update the average value size hint if amount of read rows isn't too small size_t row_size = column.size(); if (row_size > 10) { - double current_avg_value_size = static_cast(column.byte_size()) / row_size; + double current_avg_value_size = + static_cast(column.byte_size()) / static_cast(row_size); /// Heuristic is chosen so that avg_value_size_hint increases rapidly but decreases slowly. if (current_avg_value_size > avg_value_size_hint) { diff --git a/be/src/vec/data_types/data_type.h b/be/src/vec/data_types/data_type.h index dcabc423bb86856..8796e2292a5c1be 100644 --- a/be/src/vec/data_types/data_type.h +++ b/be/src/vec/data_types/data_type.h @@ -28,6 +28,7 @@ #include #include #include +#include #include #include "common/exception.h" @@ -44,7 +45,7 @@ class PColumnMeta; enum PGenericType_TypeId : int; namespace vectorized { - +#include "common/compile_check_begin.h" class IDataType; class IColumn; class BufferWritable; @@ -58,8 +59,11 @@ class Field; using DataTypePtr = std::shared_ptr; using DataTypes = std::vector; constexpr auto SERIALIZED_MEM_SIZE_LIMIT = 256; -inline size_t upper_int32(size_t size) { - return size_t((3 + size) / 4.0); + +template +T upper_int32(T size) { + static_assert(std::is_unsigned_v); + return T(static_cast(3 + size) / 4.0); } /** Properties of data type. @@ -421,4 +425,6 @@ char* serialize_const_flag_and_row_num(const IColumn** column, char* buf, const char* deserialize_const_flag_and_row_num(const char* buf, MutableColumnPtr* column, size_t* real_have_saved_num); } // namespace vectorized + +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/vec/data_types/data_type_bitmap.cpp b/be/src/vec/data_types/data_type_bitmap.cpp index 4ab6b3abc47e64e..3dc585f0b5efe70 100644 --- a/be/src/vec/data_types/data_type_bitmap.cpp +++ b/be/src/vec/data_types/data_type_bitmap.cpp @@ -30,7 +30,7 @@ #include "vec/io/io_helper.h" namespace doris::vectorized { - +#include "common/compile_check_begin.h" // binary: const flag| row num | real saved num | size array | bitmap array // : bitmap1 size | bitmap2 size | ... // : bitmap1 | bitmap2 | ... @@ -159,7 +159,7 @@ MutableColumnPtr DataTypeBitMap::create_column() const { void DataTypeBitMap::serialize_as_stream(const BitmapValue& cvalue, BufferWritable& buf) { auto& value = const_cast(cvalue); std::string memory_buffer; - int bytesize = value.getSizeInBytes(); + size_t bytesize = value.getSizeInBytes(); memory_buffer.resize(bytesize); value.write_to(const_cast(memory_buffer.data())); write_string_binary(memory_buffer, buf); diff --git a/be/src/vec/data_types/data_type_date.h b/be/src/vec/data_types/data_type_date.h index 0d62230a9b0080f..0df23022e2b6d27 100644 --- a/be/src/vec/data_types/data_type_date.h +++ b/be/src/vec/data_types/data_type_date.h @@ -35,6 +35,7 @@ #include "vec/data_types/serde/data_type_date64_serde.h" namespace doris { +#include "common/compile_check_begin.h" namespace vectorized { class BufferWritable; class ReadBuffer; @@ -92,5 +93,5 @@ class DataTypeDate final : public DataTypeNumberBase { return std::make_shared(nesting_level); } }; - +#include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/data_types/data_type_date_time.h b/be/src/vec/data_types/data_type_date_time.h index 99507a370391e18..03a6a85657935cb 100644 --- a/be/src/vec/data_types/data_type_date_time.h +++ b/be/src/vec/data_types/data_type_date_time.h @@ -45,6 +45,7 @@ class DataTypeDateV2; } // namespace doris namespace doris::vectorized { +#include "common/compile_check_begin.h" /** DateTime stores time as unix timestamp. * The value itself is independent of time zone. @@ -143,4 +144,5 @@ constexpr bool IsTimeType = IsDateTimeType || IsDateType; template constexpr bool IsTimeV2Type = IsDateTimeV2Type || IsDateV2Type; +#include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/data_types/data_type_decimal.cpp b/be/src/vec/data_types/data_type_decimal.cpp index 35575106cdbf609..d08c8268036abb9 100644 --- a/be/src/vec/data_types/data_type_decimal.cpp +++ b/be/src/vec/data_types/data_type_decimal.cpp @@ -25,9 +25,12 @@ #include #include +#include +#include #include #include "agent/be_exec_version_manager.h" +#include "common/cast_set.h" #include "runtime/decimalv2_value.h" #include "util/string_parser.hpp" #include "vec/columns/column.h" @@ -38,11 +41,12 @@ #include "vec/common/string_buffer.hpp" #include "vec/common/typeid_cast.h" #include "vec/core/types.h" +#include "vec/data_types/data_type.h" #include "vec/io/io_helper.h" #include "vec/io/reader_buffer.h" namespace doris::vectorized { - +#include "common/compile_check_begin.h" template std::string DataTypeDecimal::do_get_name() const { std::stringstream ss; @@ -124,7 +128,9 @@ void DataTypeDecimal::to_string_batch_impl(const ColumnPtr& column_ptr, auto str = value.to_string(get_format_scale()); chars.insert(str.begin(), str.end()); } - offsets[row_num] = chars.size(); + + // cast by row, so not use cast_set for performance issue + offsets[row_num] = static_cast(chars.size()); } } @@ -157,12 +163,13 @@ int64_t DataTypeDecimal::get_uncompressed_serialized_bytes(const IColumn& col if (be_exec_version >= USE_CONST_SERDE) { auto size = sizeof(bool) + sizeof(size_t) + sizeof(size_t); auto real_need_copy_num = is_column_const(column) ? 1 : column.size(); - auto mem_size = sizeof(T) * real_need_copy_num; + auto mem_size = cast_set(sizeof(T) * real_need_copy_num); if (mem_size <= SERIALIZED_MEM_SIZE_LIMIT) { return size + mem_size; } else { return size + sizeof(size_t) + - std::max(mem_size, streamvbyte_max_compressedbytes(upper_int32(mem_size))); + std::max(cast_set(mem_size), + streamvbyte_max_compressedbytes(upper_int32(mem_size))); } } else { auto size = sizeof(T) * column.size(); @@ -170,7 +177,8 @@ int64_t DataTypeDecimal::get_uncompressed_serialized_bytes(const IColumn& col return sizeof(uint32_t) + size; } else { return sizeof(uint32_t) + sizeof(size_t) + - std::max(size, streamvbyte_max_compressedbytes(upper_int32(size))); + std::max(size, + streamvbyte_max_compressedbytes(cast_set(upper_int32(size)))); } } } @@ -183,7 +191,7 @@ char* DataTypeDecimal::serialize(const IColumn& column, char* buf, int be_exe buf = serialize_const_flag_and_row_num(&data_column, buf, &real_need_copy_num); // mem_size = real_need_copy_num * sizeof(T) - const uint32_t mem_size = real_need_copy_num * sizeof(T); + UInt32 mem_size = cast_set(real_need_copy_num * sizeof(T)); const auto* origin_data = assert_cast&>(*data_column).get_data().data(); @@ -201,7 +209,7 @@ char* DataTypeDecimal::serialize(const IColumn& column, char* buf, int be_exe } } else { // row num - const auto mem_size = column.size() * sizeof(T); + UInt32 mem_size = cast_set(column.size() * sizeof(T)); *reinterpret_cast(buf) = mem_size; buf += sizeof(uint32_t); // column data @@ -230,7 +238,7 @@ const char* DataTypeDecimal::deserialize(const char* buf, MutableColumnPtr* c buf = deserialize_const_flag_and_row_num(buf, column, &real_have_saved_num); // column data - auto mem_size = real_have_saved_num * sizeof(T); + UInt32 mem_size = cast_set(real_have_saved_num * sizeof(T)); auto& container = assert_cast*>(origin_column)->get_data(); container.resize(real_have_saved_num); if (mem_size <= SERIALIZED_MEM_SIZE_LIMIT) { @@ -289,7 +297,7 @@ template bool DataTypeDecimal::parse_from_string(const std::string& str, T* res) const { StringParser::ParseResult result = StringParser::PARSE_SUCCESS; res->value = StringParser::string_to_decimal::get_primitive_type()>( - str.c_str(), str.size(), precision, scale, &result); + str.c_str(), cast_set(str.size()), precision, scale, &result); return result == StringParser::PARSE_SUCCESS || result == StringParser::PARSE_UNDERFLOW; } diff --git a/be/src/vec/data_types/data_type_factory.cpp b/be/src/vec/data_types/data_type_factory.cpp index 388710b667a343f..369809d77f68f35 100644 --- a/be/src/vec/data_types/data_type_factory.cpp +++ b/be/src/vec/data_types/data_type_factory.cpp @@ -67,7 +67,7 @@ #include "vec/data_types/data_type_time_v2.h" namespace doris::vectorized { - +#include "common/compile_check_begin.h" DataTypePtr DataTypeFactory::create_data_type(const doris::Field& col_desc) { return create_data_type(col_desc.get_desc(), col_desc.is_nullable()); } @@ -76,7 +76,7 @@ DataTypePtr DataTypeFactory::create_data_type(const TabletColumn& col_desc, bool DataTypePtr nested = nullptr; if (col_desc.type() == FieldType::OLAP_FIELD_TYPE_AGG_STATE) { DataTypes dataTypes; - for (size_t i = 0; i < col_desc.get_subtype_count(); i++) { + for (UInt32 i = 0; i < col_desc.get_subtype_count(); i++) { dataTypes.push_back(create_data_type(col_desc.get_sub_column(i))); } nested = std::make_shared( @@ -97,7 +97,7 @@ DataTypePtr DataTypeFactory::create_data_type(const TabletColumn& col_desc, bool Strings names; dataTypes.reserve(col_size); names.reserve(col_size); - for (size_t i = 0; i < col_size; i++) { + for (UInt32 i = 0; i < col_size; i++) { dataTypes.push_back(create_data_type(col_desc.get_sub_column(i))); names.push_back(col_desc.get_sub_column(i).name()); } @@ -546,13 +546,13 @@ DataTypePtr DataTypeFactory::create_data_type(const PColumnMeta& pcolumn) { create_data_type(pcolumn.children(1))); break; case PGenericType::STRUCT: { - size_t col_size = pcolumn.children_size(); + int col_size = pcolumn.children_size(); DCHECK(col_size >= 1); DataTypes dataTypes; Strings names; dataTypes.reserve(col_size); names.reserve(col_size); - for (size_t i = 0; i < col_size; i++) { + for (int i = 0; i < col_size; i++) { dataTypes.push_back(create_data_type(pcolumn.children(i))); names.push_back(pcolumn.children(i).name()); } @@ -615,10 +615,10 @@ DataTypePtr DataTypeFactory::create_data_type(const segment_v2::ColumnMetaPB& pc create_data_type(pcolumn.children_columns(1))); } else if (pcolumn.type() == static_cast(FieldType::OLAP_FIELD_TYPE_STRUCT)) { DCHECK_GE(pcolumn.children_columns().size(), 1); - size_t col_size = pcolumn.children_columns().size(); + Int32 col_size = pcolumn.children_columns().size(); DataTypes dataTypes(col_size); Strings names(col_size); - for (size_t i = 0; i < col_size; i++) { + for (Int32 i = 0; i < col_size; i++) { dataTypes[i] = create_data_type(pcolumn.children_columns(i)); } nested = std::make_shared(dataTypes, names); diff --git a/be/src/vec/data_types/data_type_fixed_length_object.cpp b/be/src/vec/data_types/data_type_fixed_length_object.cpp index 11e56892f064fe6..3d8dd001077b302 100644 --- a/be/src/vec/data_types/data_type_fixed_length_object.cpp +++ b/be/src/vec/data_types/data_type_fixed_length_object.cpp @@ -23,10 +23,13 @@ #include #include "agent/be_exec_version_manager.h" +#include "common/cast_set.h" #include "vec/columns/column.h" #include "vec/common/assert_cast.h" +#include "vec/core/types.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" char* DataTypeFixedLengthObject::serialize(const IColumn& column, char* buf, int be_exec_version) const { @@ -62,7 +65,7 @@ char* DataTypeFixedLengthObject::serialize(const IColumn& column, char* buf, return buf; } else { // row num - const auto row_num = column.size(); + const UInt32 row_num = cast_set(column.size()); *reinterpret_cast(buf) = row_num; buf += sizeof(uint32_t); // column data diff --git a/be/src/vec/data_types/data_type_jsonb.cpp b/be/src/vec/data_types/data_type_jsonb.cpp index 102f6d4889b932a..49ec95b3b448064 100644 --- a/be/src/vec/data_types/data_type_jsonb.cpp +++ b/be/src/vec/data_types/data_type_jsonb.cpp @@ -20,11 +20,13 @@ #include #include +#include "common/cast_set.h" #include "util/jsonb_utils.h" #include "vec/columns/column_const.h" #include "vec/common/assert_cast.h" #include "vec/common/string_buffer.hpp" #include "vec/common/string_ref.h" +#include "vec/core/types.h" #include "vec/io/reader_buffer.h" namespace doris { @@ -34,7 +36,7 @@ class IColumn; } // namespace doris namespace doris::vectorized { - +#include "common/compile_check_begin.h" std::string DataTypeJsonb::to_string(const IColumn& column, size_t row_num) const { auto result = check_column_const_set_readability(column, row_num); ColumnPtr ptr = result.first; @@ -59,7 +61,8 @@ void DataTypeJsonb::to_string(const class doris::vectorized::IColumn& column, si Status DataTypeJsonb::from_string(ReadBuffer& rb, IColumn* column) const { JsonBinaryValue value; - RETURN_IF_ERROR(value.from_json_string(rb.position(), rb.count())); + // Throw exception if rb.count is large than INT32_MAX + RETURN_IF_ERROR(value.from_json_string(rb.position(), cast_set(rb.count()))); auto* column_string = static_cast(column); column_string->insert_data(value.value(), value.size()); diff --git a/be/src/vec/data_types/data_type_jsonb.h b/be/src/vec/data_types/data_type_jsonb.h index 3d681e3ce797541..4aec1ccc0417a9d 100644 --- a/be/src/vec/data_types/data_type_jsonb.h +++ b/be/src/vec/data_types/data_type_jsonb.h @@ -24,6 +24,7 @@ #include #include +#include "common/cast_set.h" #include "common/status.h" #include "runtime/define_primitive_type.h" #include "runtime/jsonb_value.h" @@ -45,6 +46,7 @@ class ReadBuffer; } // namespace doris namespace doris::vectorized { +#include "common/compile_check_begin.h" class DataTypeJsonb final : public IDataType { public: using ColumnType = ColumnString; @@ -70,8 +72,9 @@ class DataTypeJsonb final : public IDataType { virtual Field get_default() const override { std::string default_json = "null"; - JsonBinaryValue binary_val(default_json.c_str(), default_json.size()); - return JsonbField(binary_val.value(), binary_val.size()); + JsonBinaryValue binary_val(default_json.c_str(), static_cast(default_json.size())); + // Throw exception if default_json.size() is large than INT32_MAX + return JsonbField(binary_val.value(), cast_set(binary_val.size())); } Field get_field(const TExprNode& node) const override { @@ -100,4 +103,6 @@ class DataTypeJsonb final : public IDataType { private: DataTypeString data_type_string; }; + +#include "common/compile_check_end.h" } // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/data_types/data_type_nullable.cpp b/be/src/vec/data_types/data_type_nullable.cpp index 66cbcb25a9b6bc9..9f155babfaac3e3 100644 --- a/be/src/vec/data_types/data_type_nullable.cpp +++ b/be/src/vec/data_types/data_type_nullable.cpp @@ -30,6 +30,7 @@ #include #include "agent/be_exec_version_manager.h" +#include "common/cast_set.h" #include "vec/columns/column.h" #include "vec/columns/column_const.h" #include "vec/columns/column_nullable.h" @@ -37,12 +38,13 @@ #include "vec/common/assert_cast.h" #include "vec/common/string_buffer.hpp" #include "vec/core/field.h" +#include "vec/core/types.h" #include "vec/data_types/data_type.h" #include "vec/data_types/data_type_nothing.h" #include "vec/io/reader_buffer.h" namespace doris::vectorized { - +#include "common/compile_check_begin.h" DataTypeNullable::DataTypeNullable(const DataTypePtr& nested_data_type_) : nested_data_type {nested_data_type_} { if (!nested_data_type) { @@ -114,8 +116,10 @@ int64_t DataTypeNullable::get_uncompressed_serialized_bytes(const IColumn& colum if (mem_size <= SERIALIZED_MEM_SIZE_LIMIT) { size += mem_size; } else { + // Throw exception if mem_size is large than UINT32_MAX size = size + sizeof(size_t) + - std::max(mem_size, streamvbyte_max_compressedbytes(upper_int32(mem_size))); + std::max(mem_size, streamvbyte_max_compressedbytes( + cast_set(upper_int32(mem_size)))); } const auto& col = assert_cast(*data_column); size = size + nested_data_type->get_uncompressed_serialized_bytes(col.get_nested_column(), @@ -126,8 +130,10 @@ int64_t DataTypeNullable::get_uncompressed_serialized_bytes(const IColumn& colum if (size_t size = sizeof(bool) * column.size(); size <= SERIALIZED_MEM_SIZE_LIMIT) { ret += size + sizeof(uint32_t); } else { + // Throw exception if mem_size is large than UINT32_MAX ret += (sizeof(uint32_t) + sizeof(size_t) + - std::max(size, streamvbyte_max_compressedbytes(upper_int32(size)))); + std::max(size, + streamvbyte_max_compressedbytes(cast_set(upper_int32(size))))); } ret += nested_data_type->get_uncompressed_serialized_bytes( assert_cast(*column.convert_to_full_column_if_const()) @@ -151,9 +157,10 @@ char* DataTypeNullable::serialize(const IColumn& column, char* buf, int be_exec_ memcpy(buf, col.get_null_map_data().data(), mem_size); buf += mem_size; } else { + // Throw exception if mem_size is large than UINT32_MAX auto encode_size = streamvbyte_encode( reinterpret_cast(col.get_null_map_data().data()), - upper_int32(mem_size), (uint8_t*)(buf + sizeof(size_t))); + cast_set(upper_int32(mem_size)), (uint8_t*)(buf + sizeof(size_t))); *reinterpret_cast(buf) = encode_size; buf += (sizeof(size_t) + encode_size); } @@ -165,16 +172,17 @@ char* DataTypeNullable::serialize(const IColumn& column, char* buf, int be_exec_ // row num auto mem_size = col.size() * sizeof(bool); - *reinterpret_cast(buf) = mem_size; + *reinterpret_cast(buf) = static_cast(mem_size); buf += sizeof(uint32_t); // null flags if (mem_size <= SERIALIZED_MEM_SIZE_LIMIT) { memcpy(buf, col.get_null_map_data().data(), mem_size); buf += mem_size; } else { + // Throw exception if mem_size is large than UINT32_MAX auto encode_size = streamvbyte_encode( reinterpret_cast(col.get_null_map_data().data()), - upper_int32(mem_size), (uint8_t*)(buf + sizeof(size_t))); + cast_set(upper_int32(mem_size)), (uint8_t*)(buf + sizeof(size_t))); *reinterpret_cast(buf) = encode_size; buf += (sizeof(size_t) + encode_size); } @@ -200,8 +208,9 @@ const char* DataTypeNullable::deserialize(const char* buf, MutableColumnPtr* col } else { size_t encode_size = *reinterpret_cast(buf); buf += sizeof(size_t); + // Throw exception if mem_size is large than UINT32_MAX streamvbyte_decode((const uint8_t*)buf, (uint32_t*)(col->get_null_map_data().data()), - upper_int32(mem_size)); + cast_set(upper_int32(mem_size))); buf += encode_size; } // column data values @@ -221,8 +230,9 @@ const char* DataTypeNullable::deserialize(const char* buf, MutableColumnPtr* col } else { size_t encode_size = *reinterpret_cast(buf); buf += sizeof(size_t); + // Throw exception if mem_size is large than UINT32_MAX streamvbyte_decode((const uint8_t*)buf, (uint32_t*)(col->get_null_map_data().data()), - upper_int32(mem_size)); + cast_set(upper_int32(mem_size))); buf += encode_size; } // data values diff --git a/be/src/vec/data_types/data_type_number_base.cpp b/be/src/vec/data_types/data_type_number_base.cpp index 7d5c831144b9711..da34d82031cda03 100644 --- a/be/src/vec/data_types/data_type_number_base.cpp +++ b/be/src/vec/data_types/data_type_number_base.cpp @@ -30,6 +30,7 @@ #include #include "agent/be_exec_version_manager.h" +#include "common/cast_set.h" #include "gutil/strings/numbers.h" #include "runtime/large_int_value.h" #include "util/mysql_global.h" @@ -39,11 +40,12 @@ #include "vec/columns/column_vector.h" #include "vec/common/assert_cast.h" #include "vec/common/string_buffer.hpp" +#include "vec/core/types.h" #include "vec/io/io_helper.h" #include "vec/io/reader_buffer.h" namespace doris::vectorized { - +#include "common/compile_check_begin.h" template void DataTypeNumberBase::to_string(const IColumn& column, size_t row_num, BufferWritable& ostr) const { @@ -197,16 +199,20 @@ int64_t DataTypeNumberBase::get_uncompressed_serialized_bytes(const IColumn& if (mem_size <= SERIALIZED_MEM_SIZE_LIMIT) { return size + mem_size; } else { + // Throw exception if mem_size is large than UINT32_MAX return size + sizeof(size_t) + - std::max(mem_size, streamvbyte_max_compressedbytes(upper_int32(mem_size))); + std::max(mem_size, streamvbyte_max_compressedbytes( + cast_set(upper_int32(mem_size)))); } } else { auto size = sizeof(T) * column.size(); if (size <= SERIALIZED_MEM_SIZE_LIMIT) { return sizeof(uint32_t) + size; } else { + // Throw exception if mem_size is large than UINT32_MAX return sizeof(uint32_t) + sizeof(size_t) + - std::max(size, streamvbyte_max_compressedbytes(upper_int32(size))); + std::max(size, + streamvbyte_max_compressedbytes(cast_set(upper_int32(size)))); } } } @@ -229,9 +235,10 @@ char* DataTypeNumberBase::serialize(const IColumn& column, char* buf, memcpy(buf, origin_data, mem_size); return buf + mem_size; } else { - auto encode_size = - streamvbyte_encode(reinterpret_cast(origin_data), - upper_int32(mem_size), (uint8_t*)(buf + sizeof(size_t))); + // Throw exception if mem_size is large than UINT32_MAX + auto encode_size = streamvbyte_encode(reinterpret_cast(origin_data), + cast_set(upper_int32(mem_size)), + (uint8_t*)(buf + sizeof(size_t))); *reinterpret_cast(buf) = encode_size; buf += sizeof(size_t); return buf + encode_size; @@ -239,7 +246,7 @@ char* DataTypeNumberBase::serialize(const IColumn& column, char* buf, } else { // row num const auto mem_size = column.size() * sizeof(T); - *reinterpret_cast(buf) = mem_size; + *reinterpret_cast(buf) = static_cast(mem_size); buf += sizeof(uint32_t); // column data auto ptr = column.convert_to_full_column_if_const(); @@ -248,10 +255,10 @@ char* DataTypeNumberBase::serialize(const IColumn& column, char* buf, memcpy(buf, origin_data, mem_size); return buf + mem_size; } - - auto encode_size = - streamvbyte_encode(reinterpret_cast(origin_data), - upper_int32(mem_size), (uint8_t*)(buf + sizeof(size_t))); + // Throw exception if mem_size is large than UINT32_MAX + auto encode_size = streamvbyte_encode(reinterpret_cast(origin_data), + cast_set(upper_int32(mem_size)), + (uint8_t*)(buf + sizeof(size_t))); *reinterpret_cast(buf) = encode_size; buf += sizeof(size_t); return buf + encode_size; @@ -277,7 +284,7 @@ const char* DataTypeNumberBase::deserialize(const char* buf, MutableColumnPtr size_t encode_size = *reinterpret_cast(buf); buf += sizeof(size_t); streamvbyte_decode((const uint8_t*)buf, (uint32_t*)(container.data()), - upper_int32(mem_size)); + cast_set(upper_int32(mem_size))); buf = buf + encode_size; } return buf; @@ -296,7 +303,7 @@ const char* DataTypeNumberBase::deserialize(const char* buf, MutableColumnPtr size_t encode_size = *reinterpret_cast(buf); buf += sizeof(size_t); streamvbyte_decode((const uint8_t*)buf, (uint32_t*)(container.data()), - upper_int32(mem_size)); + cast_set(upper_int32(mem_size))); return buf + encode_size; } } diff --git a/be/src/vec/data_types/data_type_number_base.h b/be/src/vec/data_types/data_type_number_base.h index 0468ed80c8de037..a376b458f5133bc 100644 --- a/be/src/vec/data_types/data_type_number_base.h +++ b/be/src/vec/data_types/data_type_number_base.h @@ -30,6 +30,7 @@ #include #include +#include "common/cast_set.h" #include "common/status.h" #include "runtime/define_primitive_type.h" #include "serde/data_type_number_serde.h" @@ -51,7 +52,7 @@ struct TypeId; } // namespace doris namespace doris::vectorized { - +#include "common/compile_check_begin.h" /** Implements part of the IDataType interface, common to all numbers and for Date and DateTime. */ template @@ -188,12 +189,13 @@ class DataTypeNumberBase : public IDataType { for (int row_num = 0; row_num < size; row_num++) { auto num = is_const ? col_vec.get_element(0) : col_vec.get_element(row_num); static_cast(this)->push_number(chars, num); - offsets[row_num] = chars.size(); + // push_number can check the chars is over uint32 so use static_cast here. + offsets[row_num] = static_cast(chars.size()); } } private: bool _is_null_literal = false; }; - +#include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/data_types/data_type_object.cpp b/be/src/vec/data_types/data_type_object.cpp index 6dde7df44d5ba1a..0c795e542b0dd5a 100644 --- a/be/src/vec/data_types/data_type_object.cpp +++ b/be/src/vec/data_types/data_type_object.cpp @@ -33,6 +33,7 @@ #include "vec/columns/column_object.h" #include "vec/common/assert_cast.h" #include "vec/common/typeid_cast.h" +#include "vec/core/types.h" #include "vec/data_types/data_type.h" #include "vec/data_types/data_type_factory.hpp" #include "vec/json/path_in_data.h" @@ -44,7 +45,7 @@ class IColumn; } // namespace doris namespace doris::vectorized { - +#include "common/compile_check_begin.h" DataTypeObject::DataTypeObject(const String& schema_format_, bool is_nullable_) : schema_format(to_lower(schema_format_)), is_nullable(is_nullable_) {} bool DataTypeObject::equals(const IDataType& rhs) const { @@ -115,7 +116,8 @@ char* DataTypeObject::serialize(const IColumn& column, char* buf, int be_exec_ve type->to_pb_column_meta(&column_meta_pb); std::string meta_binary; column_meta_pb.SerializeToString(&meta_binary); - *reinterpret_cast(buf) = meta_binary.size(); + // Safe cast + *reinterpret_cast(buf) = static_cast(meta_binary.size()); buf += sizeof(uint32_t); memcpy(buf, meta_binary.data(), meta_binary.size()); buf += meta_binary.size(); @@ -124,10 +126,11 @@ char* DataTypeObject::serialize(const IColumn& column, char* buf, int be_exec_ve buf = type->serialize(entry->data.get_finalized_column(), buf, be_exec_version); } // serialize num of subcolumns - *reinterpret_cast(size_pos) = num_of_columns; + // Safe case + *reinterpret_cast(size_pos) = static_cast(num_of_columns); // serialize num of rows, only take effect when subcolumns empty if (be_exec_version >= VARIANT_SERDE) { - *reinterpret_cast(buf) = column_object.rows(); + *reinterpret_cast(buf) = static_cast(column_object.rows()); buf += sizeof(uint32_t); } @@ -183,13 +186,13 @@ const char* DataTypeObject::deserialize(const char* buf, MutableColumnPtr* colum std::string DataTypeObject::to_string(const IColumn& column, size_t row_num) const { const auto& variant = assert_cast(column); std::string res; - static_cast(variant.serialize_one_row_to_string(row_num, &res)); + static_cast(variant.serialize_one_row_to_string(cast_set(row_num), &res)); return res; } void DataTypeObject::to_string(const IColumn& column, size_t row_num, BufferWritable& ostr) const { const auto& variant = assert_cast(column); - static_cast(variant.serialize_one_row_to_string(row_num, ostr)); + static_cast(variant.serialize_one_row_to_string(cast_set(row_num), ostr)); } } // namespace doris::vectorized diff --git a/be/src/vec/data_types/data_type_quantilestate.cpp b/be/src/vec/data_types/data_type_quantilestate.cpp index 32cc60698012f5e..567cbd588040314 100644 --- a/be/src/vec/data_types/data_type_quantilestate.cpp +++ b/be/src/vec/data_types/data_type_quantilestate.cpp @@ -27,6 +27,7 @@ #include "vec/io/io_helper.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" // binary: const flag | row num | read saved num | | // : quantilestate1 size | quantilestate2 size | ... // : quantilestate1 | quantilestate2 | ... @@ -158,8 +159,7 @@ MutableColumnPtr DataTypeQuantileState::create_column() const { void DataTypeQuantileState::serialize_as_stream(const QuantileState& cvalue, BufferWritable& buf) { auto& value = const_cast(cvalue); std::string memory_buffer; - int bytesize = value.get_serialized_size(); - memory_buffer.resize(bytesize); + memory_buffer.resize(value.get_serialized_size()); value.serialize(const_cast(reinterpret_cast(memory_buffer.data()))); write_string_binary(memory_buffer, buf); } diff --git a/be/src/vec/data_types/data_type_string.cpp b/be/src/vec/data_types/data_type_string.cpp index 424cd43bd3ab575..c01f2c1f24912a2 100644 --- a/be/src/vec/data_types/data_type_string.cpp +++ b/be/src/vec/data_types/data_type_string.cpp @@ -24,9 +24,11 @@ #include #include +#include #include #include "agent/be_exec_version_manager.h" +#include "common/cast_set.h" #include "common/exception.h" #include "common/status.h" #include "vec/columns/column.h" @@ -36,10 +38,11 @@ #include "vec/common/string_buffer.hpp" #include "vec/common/string_ref.h" #include "vec/core/field.h" +#include "vec/core/types.h" #include "vec/io/reader_buffer.h" namespace doris::vectorized { - +#include "common/compile_check_begin.h" std::string DataTypeString::to_string(const IColumn& column, size_t row_num) const { auto result = check_column_const_set_readability(column, row_num); ColumnPtr ptr = result.first; @@ -97,8 +100,10 @@ int64_t DataTypeString::get_uncompressed_serialized_bytes(const IColumn& column, if (offsets_size <= SERIALIZED_MEM_SIZE_LIMIT) { size += offsets_size; } else { - size += sizeof(size_t) + std::max(offsets_size, streamvbyte_max_compressedbytes( - upper_int32(offsets_size))); + // Throw exception if offsets_size is large than UINT32_MAX + size += sizeof(size_t) + + std::max(offsets_size, streamvbyte_max_compressedbytes( + cast_set(upper_int32(offsets_size)))); } size += sizeof(size_t); if (size_t bytes = data_column.get_chars().size(); bytes <= SERIALIZED_MEM_SIZE_LIMIT) { @@ -110,7 +115,8 @@ int64_t DataTypeString::get_uncompressed_serialized_bytes(const IColumn& column, "LZ4_MAX_INPUT_SIZE={}", bytes, LZ4_MAX_INPUT_SIZE); } - size += sizeof(size_t) + std::max(bytes, (size_t)LZ4_compressBound(bytes)); + size += sizeof(size_t) + + std::max(bytes, (size_t)LZ4_compressBound(cast_set(bytes))); } return size; } else { @@ -121,14 +127,18 @@ int64_t DataTypeString::get_uncompressed_serialized_bytes(const IColumn& column, offsets_size <= SERIALIZED_MEM_SIZE_LIMIT) { size += offsets_size; } else { - size += sizeof(size_t) + std::max(offsets_size, streamvbyte_max_compressedbytes( - upper_int32(offsets_size))); + // Throw exception if offsets_size is large than UINT32_MAX + size += sizeof(size_t) + + std::max(offsets_size, streamvbyte_max_compressedbytes( + cast_set(upper_int32(offsets_size)))); } if (auto bytes = data_column.get_chars().size(); bytes <= SERIALIZED_MEM_SIZE_LIMIT) { size += bytes; } else { - size += sizeof(size_t) + std::max(bytes, (size_t)LZ4_compressBound(bytes)); + // Throw exception if bytes is large than UINT32_MAX + size += sizeof(size_t) + + std::max(bytes, (size_t)LZ4_compressBound(cast_set(bytes))); } return size; } @@ -141,16 +151,17 @@ char* DataTypeString::serialize(const IColumn& column, char* buf, int be_exec_ve buf = serialize_const_flag_and_row_num(&data_column, buf, &real_need_copy_num); // mem_size = real_row_num * sizeof(IColumn::Offset) - auto mem_size = real_need_copy_num * sizeof(IColumn::Offset); + size_t mem_size = real_need_copy_num * sizeof(IColumn::Offset); const auto& string_column = assert_cast(*data_column); // offsets if (mem_size <= SERIALIZED_MEM_SIZE_LIMIT) { memcpy(buf, string_column.get_offsets().data(), mem_size); buf += mem_size; } else { + // Throw exception if mem_size is large than UINT32_MAX auto encode_size = streamvbyte_encode( reinterpret_cast(string_column.get_offsets().data()), - upper_int32(mem_size), (uint8_t*)(buf + sizeof(size_t))); + cast_set(upper_int32(mem_size)), (uint8_t*)(buf + sizeof(size_t))); *reinterpret_cast(buf) = encode_size; buf += (sizeof(size_t) + encode_size); } @@ -163,9 +174,9 @@ char* DataTypeString::serialize(const IColumn& column, char* buf, int be_exec_ve memcpy(buf, string_column.get_chars().data(), value_len); buf += value_len; } else { - auto encode_size = - LZ4_compress_fast(string_column.get_chars().raw_data(), (buf + sizeof(size_t)), - value_len, LZ4_compressBound(value_len), 1); + auto encode_size = LZ4_compress_fast(string_column.get_chars().raw_data(), + (buf + sizeof(size_t)), cast_set(value_len), + LZ4_compressBound(cast_set(value_len)), 1); *reinterpret_cast(buf) = encode_size; buf += (sizeof(size_t) + encode_size); } @@ -175,17 +186,18 @@ char* DataTypeString::serialize(const IColumn& column, char* buf, int be_exec_ve const auto& data_column = assert_cast(*ptr.get()); // row num - uint32_t mem_size = data_column.size() * sizeof(IColumn::Offset); - *reinterpret_cast(buf) = mem_size; + size_t mem_size = data_column.size() * sizeof(IColumn::Offset); + *reinterpret_cast(buf) = static_cast(mem_size); buf += sizeof(uint32_t); // offsets if (mem_size <= SERIALIZED_MEM_SIZE_LIMIT) { memcpy(buf, data_column.get_offsets().data(), mem_size); buf += mem_size; } else { + // Throw exception if mem_size is large than UINT32_MAX auto encode_size = streamvbyte_encode( reinterpret_cast(data_column.get_offsets().data()), - upper_int32(mem_size), (uint8_t*)(buf + sizeof(size_t))); + cast_set(upper_int32(mem_size)), (uint8_t*)(buf + sizeof(size_t))); *reinterpret_cast(buf) = encode_size; buf += (sizeof(size_t) + encode_size); } @@ -199,9 +211,9 @@ char* DataTypeString::serialize(const IColumn& column, char* buf, int be_exec_ve buf += value_len; return buf; } - auto encode_size = - LZ4_compress_fast(data_column.get_chars().raw_data(), (buf + sizeof(size_t)), - value_len, LZ4_compressBound(value_len), 1); + auto encode_size = LZ4_compress_fast(data_column.get_chars().raw_data(), + (buf + sizeof(size_t)), cast_set(value_len), + LZ4_compressBound(cast_set(value_len)), 1); *reinterpret_cast(buf) = encode_size; buf += (sizeof(size_t) + encode_size); return buf; @@ -229,7 +241,7 @@ const char* DataTypeString::deserialize(const char* buf, MutableColumnPtr* colum size_t encode_size = *reinterpret_cast(buf); buf += sizeof(size_t); streamvbyte_decode((const uint8_t*)buf, (uint32_t*)(offsets.data()), - upper_int32(mem_size)); + cast_set(upper_int32(mem_size))); buf += encode_size; } @@ -245,7 +257,8 @@ const char* DataTypeString::deserialize(const char* buf, MutableColumnPtr* colum } else { size_t encode_size = *reinterpret_cast(buf); buf += sizeof(size_t); - LZ4_decompress_safe(buf, reinterpret_cast(data.data()), encode_size, value_len); + LZ4_decompress_safe(buf, reinterpret_cast(data.data()), + cast_set(encode_size), cast_set(value_len)); buf += encode_size; } return buf; @@ -265,7 +278,7 @@ const char* DataTypeString::deserialize(const char* buf, MutableColumnPtr* colum size_t encode_size = *reinterpret_cast(buf); buf += sizeof(size_t); streamvbyte_decode((const uint8_t*)buf, (uint32_t*)(offsets.data()), - upper_int32(mem_size)); + cast_set(upper_int32(mem_size))); buf += encode_size; } // total length @@ -280,7 +293,8 @@ const char* DataTypeString::deserialize(const char* buf, MutableColumnPtr* colum } else { size_t encode_size = *reinterpret_cast(buf); buf += sizeof(size_t); - LZ4_decompress_safe(buf, reinterpret_cast(data.data()), encode_size, value_len); + LZ4_decompress_safe(buf, reinterpret_cast(data.data()), + cast_set(encode_size), cast_set(value_len)); buf += encode_size; } return buf; diff --git a/be/src/vec/data_types/data_type_time_v2.cpp b/be/src/vec/data_types/data_type_time_v2.cpp index 53560fac4bab2d9..604518616fabf51 100644 --- a/be/src/vec/data_types/data_type_time_v2.cpp +++ b/be/src/vec/data_types/data_type_time_v2.cpp @@ -37,6 +37,7 @@ #include "vec/runtime/vdatetime_value.h" namespace doris { +#include "common/compile_check_begin.h" namespace vectorized { class IColumn; } // namespace vectorized diff --git a/be/src/vec/data_types/data_type_time_v2.h b/be/src/vec/data_types/data_type_time_v2.h index 7688a04a9a86f72..e9f3fd383658ace 100644 --- a/be/src/vec/data_types/data_type_time_v2.h +++ b/be/src/vec/data_types/data_type_time_v2.h @@ -24,6 +24,7 @@ #include #include +#include #include #include @@ -49,7 +50,7 @@ class IColumn; } // namespace doris namespace doris::vectorized { - +#include "common/compile_check_begin.h" /** * Use UInt32 as underlying type to represent DateV2 type. * Specifically, a dateV2 type is represented as (YYYY (23 bits), MM (4 bits), dd (5 bits)). @@ -73,7 +74,8 @@ class DataTypeDateV2 final : public DataTypeNumberBase { Field get_field(const TExprNode& node) const override { DateV2Value value; - if (value.from_date_str(node.date_literal.value.c_str(), node.date_literal.value.size())) { + if (value.from_date_str(node.date_literal.value.c_str(), + cast_set(node.date_literal.value.size()))) { return value.to_date_int_val(); } else { throw doris::Exception(doris::ErrorCode::INVALID_ARGUMENT, @@ -150,8 +152,8 @@ class DataTypeDateTimeV2 final : public DataTypeNumberBase { DateV2Value value; const int32_t scale = node.type.types.empty() ? -1 : node.type.types.front().scalar_type.scale; - if (value.from_date_str(node.date_literal.value.c_str(), node.date_literal.value.size(), - scale)) { + if (value.from_date_str(node.date_literal.value.c_str(), + cast_set(node.date_literal.value.size()), scale)) { return value.to_date_int_val(); } else { throw doris::Exception(doris::ErrorCode::INVALID_ARGUMENT, @@ -181,5 +183,5 @@ template constexpr bool IsDataTypeDateTimeV2 = false; template <> inline constexpr bool IsDataTypeDateTimeV2 = true; - +#include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/runtime/vdatetime_value.cpp b/be/src/vec/runtime/vdatetime_value.cpp index 797fee1a50db157..f7e72efa0899ea8 100644 --- a/be/src/vec/runtime/vdatetime_value.cpp +++ b/be/src/vec/runtime/vdatetime_value.cpp @@ -22,6 +22,7 @@ #include #include +#include #include #include #include @@ -92,11 +93,11 @@ bool VecDateTimeValue::check_date(uint32_t year, uint32_t month, uint32_t day) { // The interval format is that with no delimiters // YYYY-MM-DD HH-MM-DD.FFFFFF AM in default format // 0 1 2 3 4 5 6 7 -bool VecDateTimeValue::from_date_str(const char* date_str, int len) { +bool VecDateTimeValue::from_date_str(const char* date_str, size_t len) { return from_date_str_base(date_str, len, nullptr); } //parse timezone to get offset -bool VecDateTimeValue::from_date_str(const char* date_str, int len, +bool VecDateTimeValue::from_date_str(const char* date_str, size_t len, const cctz::time_zone& local_time_zone) { return from_date_str_base(date_str, len, &local_time_zone); } @@ -3412,7 +3413,7 @@ const char* DateV2Value::day_name() const { template void DateV2Value::unchecked_set_time(uint16_t year, uint8_t month, uint8_t day, uint8_t hour, - uint8_t minute, uint8_t second, uint32_t microsecond) { + uint8_t minute, uint16_t second, uint32_t microsecond) { date_v2_value_.year_ = year; date_v2_value_.month_ = month; date_v2_value_.day_ = day; @@ -3425,7 +3426,7 @@ void DateV2Value::unchecked_set_time(uint16_t year, uint8_t month, uint8_t da } template -void DateV2Value::unchecked_set_time(uint8_t hour, uint8_t minute, uint8_t second, +void DateV2Value::unchecked_set_time(uint8_t hour, uint8_t minute, uint16_t second, uint32_t microsecond) { if constexpr (is_datetime) { date_v2_value_.hour_ = hour; diff --git a/be/src/vec/runtime/vdatetime_value.h b/be/src/vec/runtime/vdatetime_value.h index 9bb8916710299f1..2cf7636347851ad 100644 --- a/be/src/vec/runtime/vdatetime_value.h +++ b/be/src/vec/runtime/vdatetime_value.h @@ -32,6 +32,7 @@ #include #include +#include "gutil/integral_types.h" #include "util/hash_util.hpp" #include "util/time_lut.h" #include "util/timezone_utils.h" @@ -371,8 +372,8 @@ class VecDateTimeValue { // Now this type is a temp solution with little changes // 'YYMMDD', 'YYYYMMDD', 'YYMMDDHHMMSS', 'YYYYMMDDHHMMSS' // 'YY-MM-DD', 'YYYY-MM-DD', 'YY-MM-DD HH.MM.SS' // 'YYYYMMDDTHHMMSS' - bool from_date_str(const char* str, int len); - bool from_date_str(const char* str, int len, const cctz::time_zone& local_time_zone); + bool from_date_str(const char* str, size_t len); + bool from_date_str(const char* str, size_t len, const cctz::time_zone& local_time_zone); // Construct Date/Datetime type value from int64_t value. // Return true if convert success. Otherwise return false. @@ -428,15 +429,16 @@ class VecDateTimeValue { // Now this type is a temp solution with little changes int64_t daynr() const { return calc_daynr(_year, _month, _day); } - int year() const { return _year; } - int month() const { return _month; } + uint16_t year() const { return _year; } + uint8_t month() const { return _month; } int quarter() const { return (_month - 1) / 3 + 1; } int week() const { return week(mysql_week_mode(0)); } //00-53 - int day() const { return _day; } - int hour() const { return _hour; } - int minute() const { return _minute; } - int second() const { return _second; } - int neg() const { return _neg; } + uint8_t day() const { return _day; } + uint8_t hour() const { return _hour; } + uint8_t minute() const { return _minute; } + uint16_t second() const { return _second; } + uint16_t neg() const { return _neg; } + int64_t time_part_to_seconds() const { return _hour * SECOND_PER_HOUR + _minute * SECOND_PER_MINUTE + _second; } @@ -888,9 +890,9 @@ class DateV2Value { } void unchecked_set_time(uint16_t year, uint8_t month, uint8_t day, uint8_t hour, uint8_t minute, - uint8_t second, uint32_t microsecond = 0); + uint16_t second, uint32_t microsecond = 0); - void unchecked_set_time(uint8_t hour, uint8_t minute, uint8_t second, uint32_t microsecond); + void unchecked_set_time(uint8_t hour, uint8_t minute, uint16_t second, uint32_t microsecond); int64_t daynr() const { return calc_daynr(date_v2_value_.year_, date_v2_value_.month_, date_v2_value_.day_); From 65ac74564d7b730df9fc963d94bc68ce86cce6e9 Mon Sep 17 00:00:00 2001 From: morrySnow Date: Thu, 21 Nov 2024 21:14:21 +0800 Subject: [PATCH 023/110] [chore](Nereids) remove useless exception (#44356) - DialectTransformException - DoNotFallbackException - UnsupportedDialectException - TransformException - MetaNotFoundException --- .../catalog/constraint/TableIdentifier.java | 8 +- .../exceptions/DialectTransformException.java | 28 ------- .../exceptions/DoNotFallbackException.java | 27 ------- .../exceptions/MetaNotFoundException.java | 74 ------------------- .../exceptions/TransformException.java | 28 ------- .../UnsupportedDialectException.java | 35 --------- .../doris/nereids/rules/AppliedAwareRule.java | 3 +- .../org/apache/doris/nereids/rules/Rule.java | 3 +- 8 files changed, 6 insertions(+), 200 deletions(-) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/DialectTransformException.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/DoNotFallbackException.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/MetaNotFoundException.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/TransformException.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/UnsupportedDialectException.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/TableIdentifier.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/TableIdentifier.java index 8e510ec7a93ff54..ccf688663d2bd15 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/TableIdentifier.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/constraint/TableIdentifier.java @@ -21,7 +21,7 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.TableIf; import org.apache.doris.datasource.CatalogIf; -import org.apache.doris.nereids.exceptions.MetaNotFoundException; +import org.apache.doris.nereids.exceptions.AnalysisException; import com.google.common.base.Preconditions; import com.google.gson.annotations.SerializedName; @@ -48,15 +48,15 @@ public TableIdentifier(TableIf tableIf) { public TableIf toTableIf() { CatalogIf catalogIf = Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogId); if (catalogIf == null) { - throw new MetaNotFoundException(String.format("Can not find catalog %s in constraint", catalogId)); + throw new AnalysisException(String.format("Can not find catalog %s in constraint", catalogId)); } DatabaseIf databaseIf = catalogIf.getDbNullable(databaseId); if (databaseIf == null) { - throw new MetaNotFoundException(String.format("Can not find database %s in constraint", databaseId)); + throw new AnalysisException(String.format("Can not find database %s in constraint", databaseId)); } TableIf tableIf = databaseIf.getTableNullable(tableId); if (tableIf == null) { - throw new MetaNotFoundException(String.format("Can not find table %s in constraint", databaseId)); + throw new AnalysisException(String.format("Can not find table %s in constraint", databaseId)); } return tableIf; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/DialectTransformException.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/DialectTransformException.java deleted file mode 100644 index 3d96e6dd039898d..000000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/DialectTransformException.java +++ /dev/null @@ -1,28 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.nereids.exceptions; - -/** - * DialectTransformException when have not supported transforming for dialect converters. - */ -public class DialectTransformException extends UnsupportedOperationException { - - public DialectTransformException(String msg) { - super(String.format("Unsupported dialect transformation is %s", msg)); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/DoNotFallbackException.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/DoNotFallbackException.java deleted file mode 100644 index b6253f52c6b5df9..000000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/DoNotFallbackException.java +++ /dev/null @@ -1,27 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.nereids.exceptions; - -/** - * Exception for can not fall back error in Nereids. - */ -public class DoNotFallbackException extends RuntimeException { - public DoNotFallbackException(String msg) { - super(msg); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/MetaNotFoundException.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/MetaNotFoundException.java deleted file mode 100644 index f7d19c3f844ddd5..000000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/MetaNotFoundException.java +++ /dev/null @@ -1,74 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.nereids.exceptions; - -import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; - -import java.util.Optional; - -/** Nereids's AnalysisException. */ -public class MetaNotFoundException extends RuntimeException { - private final String message; - private final Optional line; - private final Optional startPosition; - private final Optional plan; - - public MetaNotFoundException(String message, Throwable cause, Optional line, - Optional startPosition, Optional plan) { - super(message, cause); - this.message = message; - this.line = line; - this.startPosition = startPosition; - this.plan = plan; - } - - public MetaNotFoundException(String message, Optional line, - Optional startPosition, Optional plan) { - super(message); - this.message = message; - this.line = line; - this.startPosition = startPosition; - this.plan = plan; - } - - public MetaNotFoundException(String message, Throwable cause) { - this(message, cause, Optional.empty(), Optional.empty(), Optional.empty()); - } - - public MetaNotFoundException(String message) { - this(message, Optional.empty(), Optional.empty(), Optional.empty()); - } - - @Override - public String getMessage() { - String planAnnotation = plan.map(p -> ";\n" + p.treeString()).orElse(""); - return getSimpleMessage() + planAnnotation; - } - - private String getSimpleMessage() { - if (line.isPresent() || startPosition.isPresent()) { - String lineAnnotation = line.map(l -> "line " + l).orElse(""); - String positionAnnotation = startPosition.map(s -> " pos " + s).orElse(""); - return message + ";" + lineAnnotation + positionAnnotation; - } else { - return message; - } - } - - // TODO: support ErrorCode -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/TransformException.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/TransformException.java deleted file mode 100644 index 401fdd56bab94e2..000000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/TransformException.java +++ /dev/null @@ -1,28 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.nereids.exceptions; - -/** - * All exceptions thrown by transform action in {@link org.apache.doris.nereids.rules.Rule} - * should be a subclass of this class. - */ -public class TransformException extends RuntimeException { - public TransformException(String msg) { - super(String.format("Transform error: %s", msg)); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/UnsupportedDialectException.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/UnsupportedDialectException.java deleted file mode 100644 index cdf7944c61c1583..000000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/UnsupportedDialectException.java +++ /dev/null @@ -1,35 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.nereids.exceptions; - -import org.apache.doris.nereids.parser.Dialect; - -/** - * UnsupportedDialectException when not match any in - * {@link Dialect}. - */ -public class UnsupportedDialectException extends UnsupportedOperationException { - - public UnsupportedDialectException(Dialect dialect) { - super(String.format("Unsupported dialect name is %s", dialect.getDialectName())); - } - - public UnsupportedDialectException(String type, String msg) { - super(String.format("Unsupported dialect type is %s, msg is %s", type, msg)); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/AppliedAwareRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/AppliedAwareRule.java index 8f7ea106236b5d8..5f4822ead04be3f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/AppliedAwareRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/AppliedAwareRule.java @@ -18,7 +18,6 @@ package org.apache.doris.nereids.rules; import org.apache.doris.nereids.CascadesContext; -import org.apache.doris.nereids.exceptions.TransformException; import org.apache.doris.nereids.pattern.Pattern; import org.apache.doris.nereids.pattern.ProxyPattern; import org.apache.doris.nereids.trees.plans.Plan; @@ -52,7 +51,7 @@ private AppliedAwareRule(Rule rule, BiPredicate matchRootPredicate) } @Override - public List transform(Plan plan, CascadesContext context) throws TransformException { + public List transform(Plan plan, CascadesContext context) { return rule.transform(plan, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/Rule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/Rule.java index 7d5b4001d9ae8cf..40b6225e98f4336 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/Rule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/Rule.java @@ -18,7 +18,6 @@ package org.apache.doris.nereids.rules; import org.apache.doris.nereids.CascadesContext; -import org.apache.doris.nereids.exceptions.TransformException; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.pattern.Pattern; import org.apache.doris.nereids.rules.RuleType.RuleTypeClass; @@ -73,7 +72,7 @@ public String toString() { return getRuleType().toString(); } - public abstract List transform(Plan node, CascadesContext context) throws TransformException; + public abstract List transform(Plan node, CascadesContext context); /** callback this function when the traverse framework accept a new plan which produce by this rule */ public void acceptPlan(Plan plan) { From fc549eca51d5dc16dc254e373a438c0971af14f5 Mon Sep 17 00:00:00 2001 From: airborne12 Date: Thu, 21 Nov 2024 21:34:44 +0800 Subject: [PATCH 024/110] [opt](primary key bf) enhance primary key bloomfilter by fixed slice type (#44397) Problem Summary: Currently, the primary key Bloom filter index can be created with any data type. However, when adding values, it only supports slice values. This inconsistency may lead to potential misuse or future issues. --- be/src/olap/primary_key_index.cpp | 4 +- .../segment_v2/bloom_filter_index_writer.cpp | 17 +++ .../segment_v2/bloom_filter_index_writer.h | 2 + .../bloom_filter_index_reader_writer_test.cpp | 107 +++++++++++++----- 4 files changed, 99 insertions(+), 31 deletions(-) diff --git a/be/src/olap/primary_key_index.cpp b/be/src/olap/primary_key_index.cpp index e416639cfb06cdb..5f7bedb01fc8de0 100644 --- a/be/src/olap/primary_key_index.cpp +++ b/be/src/olap/primary_key_index.cpp @@ -50,8 +50,8 @@ Status PrimaryKeyIndexBuilder::init() { auto opt = segment_v2::BloomFilterOptions(); opt.fpp = 0.01; - _bloom_filter_index_builder.reset( - new segment_v2::PrimaryKeyBloomFilterIndexWriterImpl(opt, type_info)); + RETURN_IF_ERROR(segment_v2::PrimaryKeyBloomFilterIndexWriterImpl::create( + opt, type_info, &_bloom_filter_index_builder)); return Status::OK(); } diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp index 98669ccb141ae72..edc6102703f4927 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp +++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp @@ -348,5 +348,22 @@ Status NGramBloomFilterIndexWriterImpl::create(const BloomFilterOptions& bf_opti return Status::OK(); } +Status PrimaryKeyBloomFilterIndexWriterImpl::create(const BloomFilterOptions& bf_options, + const TypeInfo* typeinfo, + std::unique_ptr* res) { + FieldType type = typeinfo->type(); + switch (type) { + case FieldType::OLAP_FIELD_TYPE_CHAR: + case FieldType::OLAP_FIELD_TYPE_VARCHAR: + case FieldType::OLAP_FIELD_TYPE_STRING: + *res = std::make_unique(bf_options, typeinfo); + break; + default: + return Status::NotSupported("unsupported type for primary key bloom filter index:{}", + std::to_string(int(type))); + } + return Status::OK(); +} + } // namespace segment_v2 } // namespace doris diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.h b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.h index 2cdf7171e3e276e..a94982438f651a7 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.h +++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.h @@ -85,6 +85,8 @@ class PrimaryKeyBloomFilterIndexWriterImpl : public BloomFilterIndexWriter { } }; + static Status create(const BloomFilterOptions& bf_options, const TypeInfo* typeinfo, + std::unique_ptr* res); // This method may allocate large memory for bf, will return error // when memory is exhaused to prevent oom. Status add_values(const void* values, size_t count) override; diff --git a/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp b/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp index 258dd9a5ff8b516..69cb343f04bf915 100644 --- a/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp +++ b/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp @@ -59,40 +59,46 @@ class BloomFilterIndexReaderWriterTest : public testing::Test { }; template -void write_bloom_filter_index_file(const std::string& file_name, const void* values, - size_t value_count, size_t null_count, - ColumnIndexMetaPB* index_meta) { +Status write_bloom_filter_index_file(const std::string& file_name, const void* values, + size_t value_count, size_t null_count, + ColumnIndexMetaPB* index_meta, + bool use_primary_key_bloom_filter = false) { const auto* type_info = get_scalar_type_info(); using CppType = typename CppTypeTraits::CppType; std::string fname = dname + "/" + file_name; auto fs = io::global_local_filesystem(); { io::FileWriterPtr file_writer; - Status st = fs->create_file(fname, &file_writer); - EXPECT_TRUE(st.ok()) << st.to_string(); + RETURN_IF_ERROR(fs->create_file(fname, &file_writer)); std::unique_ptr bloom_filter_index_writer; BloomFilterOptions bf_options; - static_cast( - BloomFilterIndexWriter::create(bf_options, type_info, &bloom_filter_index_writer)); + + if (use_primary_key_bloom_filter) { + RETURN_IF_ERROR(PrimaryKeyBloomFilterIndexWriterImpl::create( + bf_options, type_info, &bloom_filter_index_writer)); + } else { + RETURN_IF_ERROR(BloomFilterIndexWriter::create(bf_options, type_info, + &bloom_filter_index_writer)); + } + const CppType* vals = (const CppType*)values; for (int i = 0; i < value_count;) { size_t num = std::min(1024, (int)value_count - i); - static_cast(bloom_filter_index_writer->add_values(vals + i, num)); + RETURN_IF_ERROR(bloom_filter_index_writer->add_values(vals + i, num)); if (i == 2048) { // second page bloom_filter_index_writer->add_nulls(null_count); } - st = bloom_filter_index_writer->flush(); - EXPECT_TRUE(st.ok()); + RETURN_IF_ERROR(bloom_filter_index_writer->flush()); i += 1024; } - st = bloom_filter_index_writer->finish(file_writer.get(), index_meta); - EXPECT_TRUE(st.ok()) << "writer finish status:" << st.to_string(); + RETURN_IF_ERROR(bloom_filter_index_writer->finish(file_writer.get(), index_meta)); EXPECT_TRUE(file_writer->close().ok()); EXPECT_EQ(BLOOM_FILTER_INDEX, index_meta->type()); EXPECT_EQ(bf_options.strategy, index_meta->bloom_filter_index().hash_strategy()); } + return Status::OK(); } void get_bloom_filter_reader_iter(const std::string& file_name, const ColumnIndexMetaPB& meta, @@ -110,13 +116,14 @@ void get_bloom_filter_reader_iter(const std::string& file_name, const ColumnInde } template -void test_bloom_filter_index_reader_writer_template( +Status test_bloom_filter_index_reader_writer_template( const std::string file_name, typename TypeTraits::CppType* val, size_t num, size_t null_num, typename TypeTraits::CppType* not_exist_value, - bool is_slice_type = false) { + bool is_slice_type = false, bool use_primary_key_bloom_filter = false) { using CppType = typename TypeTraits::CppType; ColumnIndexMetaPB meta; - write_bloom_filter_index_file(file_name, val, num, null_num, &meta); + RETURN_IF_ERROR(write_bloom_filter_index_file(file_name, val, num, null_num, &meta, + use_primary_key_bloom_filter)); { BloomFilterIndexReader* reader = nullptr; std::unique_ptr iter; @@ -124,8 +131,7 @@ void test_bloom_filter_index_reader_writer_template( // page 0 std::unique_ptr bf; - auto st = iter->read_bloom_filter(0, &bf); - EXPECT_TRUE(st.ok()); + RETURN_IF_ERROR(iter->read_bloom_filter(0, &bf)); for (int i = 0; i < 1024; ++i) { if (is_slice_type) { Slice* value = (Slice*)(val + i); @@ -136,8 +142,7 @@ void test_bloom_filter_index_reader_writer_template( } // page 1 - st = iter->read_bloom_filter(1, &bf); - EXPECT_TRUE(st.ok()); + RETURN_IF_ERROR(iter->read_bloom_filter(1, &bf)); for (int i = 1024; i < 2048; ++i) { if (is_slice_type) { Slice* value = (Slice*)(val + i); @@ -148,8 +153,7 @@ void test_bloom_filter_index_reader_writer_template( } // page 2 - st = iter->read_bloom_filter(2, &bf); - EXPECT_TRUE(st.ok()); + RETURN_IF_ERROR(iter->read_bloom_filter(2, &bf)); for (int i = 2048; i < 3071; ++i) { if (is_slice_type) { Slice* value = (Slice*)(val + i); @@ -163,6 +167,7 @@ void test_bloom_filter_index_reader_writer_template( delete reader; } + return Status::OK(); } TEST_F(BloomFilterIndexReaderWriterTest, test_int) { @@ -175,8 +180,9 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_int) { std::string file_name = "bloom_filter_int"; int not_exist_value = 18888; - test_bloom_filter_index_reader_writer_template( + auto st = test_bloom_filter_index_reader_writer_template( file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); delete[] val; } @@ -190,8 +196,9 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_bigint) { std::string file_name = "bloom_filter_bigint"; int64_t not_exist_value = 18888; - test_bloom_filter_index_reader_writer_template( + auto st = test_bloom_filter_index_reader_writer_template( file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); delete[] val; } @@ -205,8 +212,9 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_largeint) { std::string file_name = "bloom_filter_largeint"; int128_t not_exist_value = 18888; - test_bloom_filter_index_reader_writer_template( + auto st = test_bloom_filter_index_reader_writer_template( file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); delete[] val; } @@ -224,8 +232,9 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_varchar_type) { } std::string file_name = "bloom_filter_varchar"; Slice not_exist_value("value_not_exist"); - test_bloom_filter_index_reader_writer_template( + auto st = test_bloom_filter_index_reader_writer_template( file_name, slices, num, 1, ¬_exist_value, true); + EXPECT_TRUE(st.ok()); delete[] val; delete[] slices; } @@ -244,8 +253,9 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_char) { } std::string file_name = "bloom_filter_char"; Slice not_exist_value("char_value_not_exist"); - test_bloom_filter_index_reader_writer_template( + auto st = test_bloom_filter_index_reader_writer_template( file_name, slices, num, 1, ¬_exist_value, true); + EXPECT_TRUE(st.ok()); delete[] val; delete[] slices; } @@ -260,8 +270,9 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_date) { std::string file_name = "bloom_filter_date"; uint24_t not_exist_value = 18888; - test_bloom_filter_index_reader_writer_template( + auto st = test_bloom_filter_index_reader_writer_template( file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); delete[] val; } @@ -275,8 +286,9 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_datetime) { std::string file_name = "bloom_filter_datetime"; int64_t not_exist_value = 18888; - test_bloom_filter_index_reader_writer_template( + auto st = test_bloom_filter_index_reader_writer_template( file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); delete[] val; } @@ -290,8 +302,45 @@ TEST_F(BloomFilterIndexReaderWriterTest, test_decimal) { std::string file_name = "bloom_filter_decimal"; decimal12_t not_exist_value = {666, 666}; - test_bloom_filter_index_reader_writer_template( + auto st = test_bloom_filter_index_reader_writer_template( file_name, val, num, 1, ¬_exist_value); + EXPECT_TRUE(st.ok()); + delete[] val; +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_primary_key_bloom_filter_index) { + size_t num = 1024 * 3 - 1; + std::vector val_strings(num); + for (size_t i = 0; i < num; ++i) { + val_strings[i] = "primary_key_" + std::to_string(i); + } + std::vector slices(num); + for (size_t i = 0; i < num; ++i) { + slices[i] = Slice(val_strings[i]); + } + + std::string file_name = "primary_key_bloom_filter_index"; + Slice not_exist_value("primary_key_not_exist"); + + auto st = test_bloom_filter_index_reader_writer_template( + file_name, slices.data(), num, 0, ¬_exist_value, true, true); + EXPECT_TRUE(st.ok()); +} + +TEST_F(BloomFilterIndexReaderWriterTest, test_primary_key_bloom_filter_index_int) { + size_t num = 1024 * 3 - 1; + int* val = new int[num]; + for (int i = 0; i < num; ++i) { + // there will be 3 bloom filter pages + val[i] = 10000 + i + 1; + } + + std::string file_name = "primary_key_bloom_filter_index_int"; + int not_exist_value = 18888; + auto st = test_bloom_filter_index_reader_writer_template( + file_name, val, num, 1, ¬_exist_value, false, true); + EXPECT_FALSE(st.ok()); + EXPECT_EQ(st.code(), TStatusCode::NOT_IMPLEMENTED_ERROR); delete[] val; } From 867bd153ad1fb1def3f25be0114bce8b3d11c616 Mon Sep 17 00:00:00 2001 From: huanghaibin Date: Thu, 21 Nov 2024 23:16:08 +0800 Subject: [PATCH 025/110] [fix](cloud-mow) Fix the issue of missing and removing some old version delete bitmap (#44300) ### What problem does this PR solve? Related PR: #40204 Problem Summary: pr #40204 support removing old delete bitmap, however it donesn't consider boundary which will lead to miss some delete bitmap should be removed on be , it only affect local delete bitmap, the delete bitmap store in fdb is right, and these missing delete bitmaps can only be deleted by next base compaciton before this pr. --- be/src/cloud/cloud_cumulative_compaction.cpp | 5 +- be/src/cloud/cloud_delete_bitmap_action.cpp | 61 ++++++- be/src/cloud/cloud_delete_bitmap_action.h | 5 +- be/src/cloud/cloud_meta_mgr.cpp | 15 +- be/src/cloud/cloud_meta_mgr.h | 5 +- be/src/olap/tablet_meta.cpp | 6 +- be/src/service/http_service.cpp | 13 +- ...ction_remove_old_version_delete_bitmap.out | 67 ++++++-- ...on_remove_old_version_delete_bitmap.groovy | 151 +++++++++++++----- 9 files changed, 253 insertions(+), 75 deletions(-) diff --git a/be/src/cloud/cloud_cumulative_compaction.cpp b/be/src/cloud/cloud_cumulative_compaction.cpp index 6b74e70ee1b4b8e..2f08082f51b5f32 100644 --- a/be/src/cloud/cloud_cumulative_compaction.cpp +++ b/be/src/cloud/cloud_cumulative_compaction.cpp @@ -393,12 +393,9 @@ Status CloudCumulativeCompaction::process_old_version_delete_bitmap() { rowset->rowset_id().to_string(); DeleteBitmap::BitmapKey start {rowset->rowset_id(), seg_id, 0}; DeleteBitmap::BitmapKey end {rowset->rowset_id(), seg_id, pre_max_version}; - DeleteBitmap::BitmapKey before_end {rowset->rowset_id(), seg_id, - pre_max_version - 1}; auto d = _tablet->tablet_meta()->delete_bitmap().get_agg( {rowset->rowset_id(), seg_id, pre_max_version}); - to_remove_vec.emplace_back( - std::make_tuple(_tablet->tablet_id(), start, before_end)); + to_remove_vec.emplace_back(std::make_tuple(_tablet->tablet_id(), start, end)); if (d->isEmpty()) { continue; } diff --git a/be/src/cloud/cloud_delete_bitmap_action.cpp b/be/src/cloud/cloud_delete_bitmap_action.cpp index 60db5896dfab8ac..86cc535e1bc88e5 100644 --- a/be/src/cloud/cloud_delete_bitmap_action.cpp +++ b/be/src/cloud/cloud_delete_bitmap_action.cpp @@ -33,6 +33,7 @@ #include #include +#include "cloud/cloud_meta_mgr.h" #include "cloud/cloud_tablet.h" #include "cloud/cloud_tablet_mgr.h" #include "common/logging.h" @@ -78,8 +79,8 @@ static Status _check_param(HttpRequest* req, uint64_t* tablet_id) { return Status::OK(); } -Status CloudDeleteBitmapAction::_handle_show_delete_bitmap_count(HttpRequest* req, - std::string* json_result) { +Status CloudDeleteBitmapAction::_handle_show_local_delete_bitmap_count(HttpRequest* req, + std::string* json_result) { uint64_t tablet_id = 0; // check & retrieve tablet_id from req if it contains RETURN_NOT_OK_STATUS_WITH_WARN(_check_param(req, &tablet_id), "check param failed"); @@ -95,6 +96,50 @@ Status CloudDeleteBitmapAction::_handle_show_delete_bitmap_count(HttpRequest* re auto count = tablet->tablet_meta()->delete_bitmap().get_delete_bitmap_count(); auto cardinality = tablet->tablet_meta()->delete_bitmap().cardinality(); auto size = tablet->tablet_meta()->delete_bitmap().get_size(); + LOG(INFO) << "show_local_delete_bitmap_count,tablet_id=" << tablet_id << ",count=" << count + << ",cardinality=" << cardinality << ",size=" << size; + + rapidjson::Document root; + root.SetObject(); + root.AddMember("delete_bitmap_count", count, root.GetAllocator()); + root.AddMember("cardinality", cardinality, root.GetAllocator()); + root.AddMember("size", size, root.GetAllocator()); + + // to json string + rapidjson::StringBuffer strbuf; + rapidjson::PrettyWriter writer(strbuf); + root.Accept(writer); + *json_result = std::string(strbuf.GetString()); + + return Status::OK(); +} + +Status CloudDeleteBitmapAction::_handle_show_ms_delete_bitmap_count(HttpRequest* req, + std::string* json_result) { + uint64_t tablet_id = 0; + // check & retrieve tablet_id from req if it contains + RETURN_NOT_OK_STATUS_WITH_WARN(_check_param(req, &tablet_id), "check param failed"); + if (tablet_id == 0) { + return Status::InternalError("check param failed: missing tablet_id"); + } + TabletMetaSharedPtr tablet_meta; + auto st = _engine.meta_mgr().get_tablet_meta(tablet_id, &tablet_meta); + if (!st.ok()) { + LOG(WARNING) << "failed to get_tablet_meta tablet=" << tablet_id + << ", st=" << st.to_string(); + return st; + } + auto tablet = std::make_shared(_engine, std::move(tablet_meta)); + st = _engine.meta_mgr().sync_tablet_rowsets(tablet.get(), false, true, true); + if (!st.ok()) { + LOG(WARNING) << "failed to sync tablet=" << tablet_id << ", st=" << st; + return st; + } + auto count = tablet->tablet_meta()->delete_bitmap().get_delete_bitmap_count(); + auto cardinality = tablet->tablet_meta()->delete_bitmap().cardinality(); + auto size = tablet->tablet_meta()->delete_bitmap().get_size(); + LOG(INFO) << "show_ms_delete_bitmap_count,tablet_id=" << tablet_id << ",count=" << count + << ",cardinality=" << cardinality << ",size=" << size; rapidjson::Document root; root.SetObject(); @@ -113,9 +158,17 @@ Status CloudDeleteBitmapAction::_handle_show_delete_bitmap_count(HttpRequest* re void CloudDeleteBitmapAction::handle(HttpRequest* req) { req->add_output_header(HttpHeaders::CONTENT_TYPE, HEADER_JSON.data()); - if (_delete_bitmap_action_type == DeleteBitmapActionType::COUNT_INFO) { + if (_delete_bitmap_action_type == DeleteBitmapActionType::COUNT_LOCAL) { + std::string json_result; + Status st = _handle_show_local_delete_bitmap_count(req, &json_result); + if (!st.ok()) { + HttpChannel::send_reply(req, HttpStatus::OK, st.to_json()); + } else { + HttpChannel::send_reply(req, HttpStatus::OK, json_result); + } + } else if (_delete_bitmap_action_type == DeleteBitmapActionType::COUNT_MS) { std::string json_result; - Status st = _handle_show_delete_bitmap_count(req, &json_result); + Status st = _handle_show_ms_delete_bitmap_count(req, &json_result); if (!st.ok()) { HttpChannel::send_reply(req, HttpStatus::OK, st.to_json()); } else { diff --git a/be/src/cloud/cloud_delete_bitmap_action.h b/be/src/cloud/cloud_delete_bitmap_action.h index 9321661374c195d..35739a7373efc87 100644 --- a/be/src/cloud/cloud_delete_bitmap_action.h +++ b/be/src/cloud/cloud_delete_bitmap_action.h @@ -31,7 +31,7 @@ class HttpRequest; class ExecEnv; -enum class DeleteBitmapActionType { COUNT_INFO = 1 }; +enum class DeleteBitmapActionType { COUNT_LOCAL = 1, COUNT_MS = 2 }; /// This action is used for viewing the delete bitmap status class CloudDeleteBitmapAction : public HttpHandlerWithAuth { @@ -45,7 +45,8 @@ class CloudDeleteBitmapAction : public HttpHandlerWithAuth { void handle(HttpRequest* req) override; private: - Status _handle_show_delete_bitmap_count(HttpRequest* req, std::string* json_result); + Status _handle_show_local_delete_bitmap_count(HttpRequest* req, std::string* json_result); + Status _handle_show_ms_delete_bitmap_count(HttpRequest* req, std::string* json_result); private: CloudStorageEngine& _engine; diff --git a/be/src/cloud/cloud_meta_mgr.cpp b/be/src/cloud/cloud_meta_mgr.cpp index 5c699ae01590502..05341d0d4bab824 100644 --- a/be/src/cloud/cloud_meta_mgr.cpp +++ b/be/src/cloud/cloud_meta_mgr.cpp @@ -385,7 +385,7 @@ Status CloudMetaMgr::get_tablet_meta(int64_t tablet_id, TabletMetaSharedPtr* tab } Status CloudMetaMgr::sync_tablet_rowsets(CloudTablet* tablet, bool warmup_delta_data, - bool sync_delete_bitmap) { + bool sync_delete_bitmap, bool full_sync) { using namespace std::chrono; TEST_SYNC_POINT_RETURN_WITH_VALUE("CloudMetaMgr::sync_tablet_rowsets", Status::OK(), tablet); @@ -411,7 +411,11 @@ Status CloudMetaMgr::sync_tablet_rowsets(CloudTablet* tablet, bool warmup_delta_ idx->set_partition_id(tablet->partition_id()); { std::shared_lock rlock(tablet->get_header_lock()); - req.set_start_version(tablet->max_version_unlocked() + 1); + if (full_sync) { + req.set_start_version(0); + } else { + req.set_start_version(tablet->max_version_unlocked() + 1); + } req.set_base_compaction_cnt(tablet->base_compaction_cnt()); req.set_cumulative_compaction_cnt(tablet->cumulative_compaction_cnt()); req.set_cumulative_point(tablet->cumulative_layer_point()); @@ -471,7 +475,7 @@ Status CloudMetaMgr::sync_tablet_rowsets(CloudTablet* tablet, bool warmup_delta_ DeleteBitmap delete_bitmap(tablet_id); int64_t old_max_version = req.start_version() - 1; auto st = sync_tablet_delete_bitmap(tablet, old_max_version, resp.rowset_meta(), - resp.stats(), req.idx(), &delete_bitmap); + resp.stats(), req.idx(), &delete_bitmap, full_sync); if (st.is() && tried++ < retry_times) { LOG_WARNING("rowset meta is expired, need to retry") .tag("tablet", tablet->tablet_id()) @@ -617,12 +621,13 @@ bool CloudMetaMgr::sync_tablet_delete_bitmap_by_cache(CloudTablet* tablet, int64 Status CloudMetaMgr::sync_tablet_delete_bitmap(CloudTablet* tablet, int64_t old_max_version, std::ranges::range auto&& rs_metas, const TabletStatsPB& stats, const TabletIndexPB& idx, - DeleteBitmap* delete_bitmap) { + DeleteBitmap* delete_bitmap, bool full_sync) { if (rs_metas.empty()) { return Status::OK(); } - if (sync_tablet_delete_bitmap_by_cache(tablet, old_max_version, rs_metas, delete_bitmap)) { + if (!full_sync && + sync_tablet_delete_bitmap_by_cache(tablet, old_max_version, rs_metas, delete_bitmap)) { return Status::OK(); } else { LOG(WARNING) << "failed to sync delete bitmap by txn info. tablet_id=" diff --git a/be/src/cloud/cloud_meta_mgr.h b/be/src/cloud/cloud_meta_mgr.h index a657c0fdd8e3500..c49b036ad90c151 100644 --- a/be/src/cloud/cloud_meta_mgr.h +++ b/be/src/cloud/cloud_meta_mgr.h @@ -58,7 +58,7 @@ class CloudMetaMgr { Status get_tablet_meta(int64_t tablet_id, std::shared_ptr* tablet_meta); Status sync_tablet_rowsets(CloudTablet* tablet, bool warmup_delta_data = false, - bool sync_delete_bitmap = true); + bool sync_delete_bitmap = true, bool full_sync = false); Status prepare_rowset(const RowsetMeta& rs_meta, std::shared_ptr* existed_rs_meta = nullptr); @@ -116,7 +116,8 @@ class CloudMetaMgr { Status sync_tablet_delete_bitmap(CloudTablet* tablet, int64_t old_max_version, std::ranges::range auto&& rs_metas, const TabletStatsPB& stats, - const TabletIndexPB& idx, DeleteBitmap* delete_bitmap); + const TabletIndexPB& idx, DeleteBitmap* delete_bitmap, + bool full_sync = false); void check_table_size_correctness(const RowsetMeta& rs_meta); int64_t get_segment_file_size(const RowsetMeta& rs_meta); int64_t get_inverted_index_file_szie(const RowsetMeta& rs_meta); diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp index 4005c818bc5023c..9a27b95dbcd446d 100644 --- a/be/src/olap/tablet_meta.cpp +++ b/be/src/olap/tablet_meta.cpp @@ -1209,9 +1209,13 @@ void DeleteBitmap::remove_stale_delete_bitmap_from_queue(const std::vector(delete_bitmap_tuple); auto end_bmk = std::get<2>(delete_bitmap_tuple); + // the key range of to be removed is [start_bmk,end_bmk), + // due to the different definitions of the right boundary, + // so use end_bmk as right boundary when removing local delete bitmap, + // use (end_bmk - 1) as right boundary when removing ms delete bitmap remove(start_bmk, end_bmk); to_delete.emplace_back(std::make_tuple(std::get<0>(start_bmk).to_string(), 0, - std::get<2>(end_bmk))); + std::get<2>(end_bmk) - 1)); } _stale_delete_bitmap.erase(version_str); } diff --git a/be/src/service/http_service.cpp b/be/src/service/http_service.cpp index e7b920796a1b985..57600d1f56aae93 100644 --- a/be/src/service/http_service.cpp +++ b/be/src/service/http_service.cpp @@ -425,11 +425,16 @@ void HttpService::register_cloud_handler(CloudStorageEngine& engine) { TPrivilegeHier::GLOBAL, TPrivilegeType::ADMIN)); _ev_http_server->register_handler(HttpMethod::GET, "/api/compaction/run_status", run_status_compaction_action); - CloudDeleteBitmapAction* count_delete_bitmap_action = - _pool.add(new CloudDeleteBitmapAction(DeleteBitmapActionType::COUNT_INFO, _env, engine, + CloudDeleteBitmapAction* count_local_delete_bitmap_action = + _pool.add(new CloudDeleteBitmapAction(DeleteBitmapActionType::COUNT_LOCAL, _env, engine, TPrivilegeHier::GLOBAL, TPrivilegeType::ADMIN)); - _ev_http_server->register_handler(HttpMethod::GET, "/api/delete_bitmap/count", - count_delete_bitmap_action); + _ev_http_server->register_handler(HttpMethod::GET, "/api/delete_bitmap/count_local", + count_local_delete_bitmap_action); + CloudDeleteBitmapAction* count_ms_delete_bitmap_action = + _pool.add(new CloudDeleteBitmapAction(DeleteBitmapActionType::COUNT_MS, _env, engine, + TPrivilegeHier::GLOBAL, TPrivilegeType::ADMIN)); + _ev_http_server->register_handler(HttpMethod::GET, "/api/delete_bitmap/count_ms", + count_ms_delete_bitmap_action); #ifdef ENABLE_INJECTION_POINT InjectionPointAction* injection_point_action = _pool.add(new InjectionPointAction); _ev_http_server->register_handler(HttpMethod::GET, "/api/injection_point/{op}", diff --git a/regression-test/data/compaction/test_cu_compaction_remove_old_version_delete_bitmap.out b/regression-test/data/compaction/test_cu_compaction_remove_old_version_delete_bitmap.out index 1c3611fe0b7506c..37dfa3b93a5878e 100644 --- a/regression-test/data/compaction/test_cu_compaction_remove_old_version_delete_bitmap.out +++ b/regression-test/data/compaction/test_cu_compaction_remove_old_version_delete_bitmap.out @@ -1,29 +1,78 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !sql -- -0 0 0 -1 8 8 +0 0 8 +1 1 1 +2 2 2 +3 3 3 +4 4 4 +5 5 5 +6 6 6 +7 7 7 +8 8 8 -- !sql -- -0 0 0 -1 8 8 +0 0 8 +1 1 1 +2 2 2 +3 3 3 +4 4 4 +5 5 5 +6 6 6 +7 7 7 +8 8 8 -- !sql -- -0 0 0 +0 0 13 1 13 13 +2 2 2 +3 3 3 +4 4 4 +5 5 5 +6 6 6 +7 7 7 +8 8 8 -- !sql -- -0 0 0 +0 0 13 1 13 13 +2 2 2 +3 3 3 +4 4 4 +5 5 5 +6 6 6 +7 7 7 +8 8 8 -- !sql -- -0 0 0 +0 0 18 1 23 23 +2 2 2 +3 3 3 +4 4 4 +5 5 5 +6 6 6 +7 7 7 +8 8 8 -- !sql -- -0 0 0 +0 0 18 1 23 23 +2 2 2 +3 3 3 +4 4 4 +5 5 5 +6 6 6 +7 7 7 +8 8 8 -- !sql -- -0 0 0 +0 5 5 1 28 28 +2 2 2 +3 3 3 +4 4 4 +5 5 5 +6 6 6 +7 7 7 +8 8 8 diff --git a/regression-test/suites/compaction/test_cu_compaction_remove_old_version_delete_bitmap.groovy b/regression-test/suites/compaction/test_cu_compaction_remove_old_version_delete_bitmap.groovy index 2219cc175b534ba..a36cb4579ca487f 100644 --- a/regression-test/suites/compaction/test_cu_compaction_remove_old_version_delete_bitmap.groovy +++ b/regression-test/suites/compaction/test_cu_compaction_remove_old_version_delete_bitmap.groovy @@ -123,11 +123,11 @@ suite("test_cu_compaction_remove_old_version_delete_bitmap", "nonConcurrent") { } while (running) } - def getDeleteBitmapStatus = { be_host, be_http_port, tablet_id -> + def getLocalDeleteBitmapStatus = { be_host, be_http_port, tablet_id -> boolean running = true StringBuilder sb = new StringBuilder(); sb.append("curl -X GET http://${be_host}:${be_http_port}") - sb.append("/api/delete_bitmap/count?tablet_id=") + sb.append("/api/delete_bitmap/count_local?tablet_id=") sb.append(tablet_id) String command = sb.toString() @@ -135,7 +135,25 @@ suite("test_cu_compaction_remove_old_version_delete_bitmap", "nonConcurrent") { process = command.execute() code = process.waitFor() out = process.getText() - logger.info("Get delete bitmap count status: =" + code + ", out=" + out) + logger.info("Get local delete bitmap count status: =" + code + ", out=" + out) + assertEquals(code, 0) + def deleteBitmapStatus = parseJson(out.trim()) + return deleteBitmapStatus + } + + def getMSDeleteBitmapStatus = { be_host, be_http_port, tablet_id -> + boolean running = true + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://${be_host}:${be_http_port}") + sb.append("/api/delete_bitmap/count_ms?tablet_id=") + sb.append(tablet_id) + + String command = sb.toString() + logger.info(command) + process = command.execute() + code = process.waitFor() + out = process.getText() + logger.info("Get ms delete bitmap count status: =" + code + ", out=" + out) assertEquals(code, 0) def deleteBitmapStatus = parseJson(out.trim()) return deleteBitmapStatus @@ -174,21 +192,24 @@ suite("test_cu_compaction_remove_old_version_delete_bitmap", "nonConcurrent") { GetDebugPoint().enableDebugPointForAllBEs("CloudCumulativeCompaction.modify_rowsets.delete_expired_stale_rowsets") // 1. test normal sql "sync" - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,1,'1'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,2,'2'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,3,'3'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,4,'4'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,5,'5'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,6,'6'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,7,'7'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,8,'8'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'1'),(1,1,'1'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'2'),(2,2,'2'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'3'),(3,3,'3'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'4'),(4,4,'4'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'5'),(5,5,'5'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'6'),(6,6,'6'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'7'),(7,7,'7'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'8'),(8,8,'8'); """ qt_sql "select * from ${testTable} order by plan_id" // trigger compaction to generate base rowset def tablets = sql_return_maparray """ show tablets from ${testTable}; """ logger.info("tablets: " + tablets) - def delete_bitmap_count = 0 + def local_delete_bitmap_count = 0 + def ms_delete_bitmap_count = 0 + def local_delete_bitmap_cardinality = 0; + def ms_delete_bitmap_cardinality = 0; for (def tablet in tablets) { String tablet_id = tablet.TabletId def tablet_info = sql_return_maparray """ show tablet ${tablet_id}; """ @@ -197,9 +218,20 @@ suite("test_cu_compaction_remove_old_version_delete_bitmap", "nonConcurrent") { getTabletStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id); // before compaction, delete_bitmap_count is (rowsets num - 1) - delete_bitmap_count = getDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).delete_bitmap_count - assertTrue(delete_bitmap_count == 7) - logger.info("delete_bitmap_count:" + delete_bitmap_count) + local_delete_bitmap_count = getLocalDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).delete_bitmap_count + ms_delete_bitmap_count = getMSDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).delete_bitmap_count + logger.info("local_delete_bitmap_count:" + local_delete_bitmap_count) + logger.info("ms_delete_bitmap_count:" + ms_delete_bitmap_count) + assertTrue(local_delete_bitmap_count == 7) + assertTrue(local_delete_bitmap_count == ms_delete_bitmap_count) + + local_delete_bitmap_cardinality = getLocalDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).cardinality + ms_delete_bitmap_cardinality = getMSDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).cardinality + logger.info("local_delete_bitmap_cardinality:" + local_delete_bitmap_cardinality) + logger.info("ms_delete_bitmap_cardinality:" + ms_delete_bitmap_cardinality) + assertTrue(local_delete_bitmap_cardinality == 7) + assertTrue(local_delete_bitmap_cardinality == ms_delete_bitmap_cardinality) + assertTrue(triggerCompaction(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], "cumulative", tablet_id).contains("Success")); @@ -211,11 +243,11 @@ suite("test_cu_compaction_remove_old_version_delete_bitmap", "nonConcurrent") { def now = System.currentTimeMillis() - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,9,'9'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,10,'10'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,11,'11'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,12,'12'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,13,'13'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'9'),(1,9,'9'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'10'),(1,10,'10'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'11'),(1,11,'11'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'12'),(1,12,'12'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'13'),(1,13,'13'); """ def time_diff = System.currentTimeMillis() - now logger.info("time_diff:" + time_diff) @@ -230,11 +262,21 @@ suite("test_cu_compaction_remove_old_version_delete_bitmap", "nonConcurrent") { def tablet_info = sql_return_maparray """ show tablet ${tablet_id}; """ logger.info("tablet: " + tablet_info) - // before compaction, delete_bitmap_count is (rowsets num - 1) + // before compaction, local delete_bitmap_count is (total rowsets num - 1), ms delete_bitmap_count is new rowset num String trigger_backend_id = tablet.BackendId - delete_bitmap_count = getDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).delete_bitmap_count - logger.info("delete_bitmap_count:" + delete_bitmap_count) - assertTrue(delete_bitmap_count == 12) + local_delete_bitmap_count = getLocalDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).delete_bitmap_count + ms_delete_bitmap_count = getMSDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).delete_bitmap_count + logger.info("local_delete_bitmap_count:" + local_delete_bitmap_count) + logger.info("ms_delete_bitmap_count:" + ms_delete_bitmap_count) + assertTrue(local_delete_bitmap_count == 12) + assertTrue(ms_delete_bitmap_count == 5) + + local_delete_bitmap_cardinality = getLocalDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).cardinality + ms_delete_bitmap_cardinality = getMSDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).cardinality + logger.info("local_delete_bitmap_cardinality:" + local_delete_bitmap_cardinality) + logger.info("ms_delete_bitmap_cardinality:" + ms_delete_bitmap_cardinality) + assertTrue(local_delete_bitmap_cardinality == 17) + assertTrue(ms_delete_bitmap_cardinality == 10) getTabletStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id); assertTrue(triggerCompaction(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], @@ -244,9 +286,19 @@ suite("test_cu_compaction_remove_old_version_delete_bitmap", "nonConcurrent") { Thread.sleep(1000) // after compaction, delete_bitmap_count is 1 - delete_bitmap_count = getDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).delete_bitmap_count - logger.info("delete_bitmap_count:" + delete_bitmap_count) - assertTrue(delete_bitmap_count == 1) + local_delete_bitmap_count = getLocalDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).delete_bitmap_count + ms_delete_bitmap_count = getMSDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).delete_bitmap_count + logger.info("local_delete_bitmap_count:" + local_delete_bitmap_count) + logger.info("ms_delete_bitmap_count:" + ms_delete_bitmap_count) + assertTrue(local_delete_bitmap_count == 1) + assertTrue(local_delete_bitmap_count == ms_delete_bitmap_count) + + local_delete_bitmap_cardinality = getLocalDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).cardinality + ms_delete_bitmap_cardinality = getMSDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).cardinality + logger.info("local_delete_bitmap_cardinality:" + local_delete_bitmap_cardinality) + logger.info("ms_delete_bitmap_cardinality:" + ms_delete_bitmap_cardinality) + assertTrue(local_delete_bitmap_cardinality == 2) + assertTrue(ms_delete_bitmap_cardinality == 2) } qt_sql "select * from ${testTable} order by plan_id" @@ -255,11 +307,11 @@ suite("test_cu_compaction_remove_old_version_delete_bitmap", "nonConcurrent") { now = System.currentTimeMillis() - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,19,'19'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,20,'20'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,21,'21'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,22,'22'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,23,'23'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'14'),(1,19,'19'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'15'),(1,20,'20'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'16'),(1,21,'21'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'17'),(1,22,'22'); """ + sql """ INSERT INTO ${testTable} VALUES (0,0,'18'),(1,23,'23'); """ time_diff = System.currentTimeMillis() - now logger.info("time_diff:" + time_diff) @@ -273,9 +325,19 @@ suite("test_cu_compaction_remove_old_version_delete_bitmap", "nonConcurrent") { logger.info("tablet: " + tablet_info) String trigger_backend_id = tablet.BackendId - delete_bitmap_count = getDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).delete_bitmap_count - assertTrue(delete_bitmap_count == 6) - logger.info("delete_bitmap_count:" + delete_bitmap_count) + local_delete_bitmap_count = getLocalDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).delete_bitmap_count + ms_delete_bitmap_count = getMSDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).delete_bitmap_count + logger.info("local_delete_bitmap_count:" + local_delete_bitmap_count) + logger.info("ms_delete_bitmap_count:" + ms_delete_bitmap_count) + assertTrue(local_delete_bitmap_count == 6) + assertTrue(local_delete_bitmap_count == ms_delete_bitmap_count) + + local_delete_bitmap_cardinality = getLocalDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).cardinality + ms_delete_bitmap_cardinality = getMSDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).cardinality + logger.info("local_delete_bitmap_cardinality:" + local_delete_bitmap_cardinality) + logger.info("ms_delete_bitmap_cardinality:" + ms_delete_bitmap_cardinality) + assertTrue(local_delete_bitmap_cardinality == 12) + assertTrue(ms_delete_bitmap_cardinality == 12) getTabletStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id); assertTrue(triggerCompaction(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], @@ -283,28 +345,29 @@ suite("test_cu_compaction_remove_old_version_delete_bitmap", "nonConcurrent") { waitForCompaction(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id) getTabletStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id); - // update fail, delete_bitmap_count will not change + // update fail, local delete_bitmap_count will not change Thread.sleep(1000) - delete_bitmap_count = getDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).delete_bitmap_count - assertTrue(delete_bitmap_count == 6) - logger.info("delete_bitmap_count:" + delete_bitmap_count) + local_delete_bitmap_count = getLocalDeleteBitmapStatus(backendId_to_backendIP[trigger_backend_id], backendId_to_backendHttpPort[trigger_backend_id], tablet_id).delete_bitmap_count + logger.info("local_delete_bitmap_count:" + local_delete_bitmap_count) + assertTrue(local_delete_bitmap_count == 6) } qt_sql "select * from ${testTable} order by plan_id" now = System.currentTimeMillis() - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,24,'24'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,25,'25'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,26,'26'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,27,'27'); """ - sql """ INSERT INTO ${testTable} VALUES (0,0,'0'),(1,28,'28'); """ + sql """ INSERT INTO ${testTable} VALUES (0,1,'1'),(1,24,'24'); """ + sql """ INSERT INTO ${testTable} VALUES (0,3,'2'),(1,25,'25'); """ + sql """ INSERT INTO ${testTable} VALUES (0,3,'3'),(1,26,'26'); """ + sql """ INSERT INTO ${testTable} VALUES (0,4,'4'),(1,27,'27'); """ + sql """ INSERT INTO ${testTable} VALUES (0,5,'5'),(1,28,'28'); """ time_diff = System.currentTimeMillis() - now logger.info("time_diff:" + time_diff) assertTrue(time_diff <= timeout, "wait_for_insert_into_values timeout") qt_sql "select * from ${testTable} order by plan_id" + GetDebugPoint().disableDebugPointForAllBEs("CloudCumulativeCompaction.modify_rowsets.update_delete_bitmap_failed") } finally { reset_be_param("compaction_promotion_version_count") From ca579c10a5ed8da4cedf2064a3c2e10056c24316 Mon Sep 17 00:00:00 2001 From: abmdocrt Date: Thu, 21 Nov 2024 23:23:42 +0800 Subject: [PATCH 026/110] [Fix](full compaction) Full compaction should not do ordered data compaction (#44359) Problem: For a duplicate table with the following distribution, if it has already completed cumulative compaction and then undergoes full compaction, it will cause a BE core issue. Check failed: new_point == Tablet::K_INVALID_CUMULATIVE_POINT || new_point >= _cumulative_point Unexpected cumulative point: 1087, origin: 2801. "rowsets": [ "[0-386] 0 DATA NONOVERLAPPING 02000000000198aabe4290f2b0f5f35610c08a233a061892 0", "[387-387] 0 DELETE OVERLAP_UNKNOWN 0200000000541310ac4d76e7580a708a2823a4d7a4f06090 0", "[388-388] 0 DELETE OVERLAP_UNKNOWN 0200000000541d76ac4d76e7580a708a2823a4d7a4f06090 0", "[389-389] 0 DELETE OVERLAP_UNKNOWN 0200000000543b4dac4d76e7580a708a2823a4d7a4f06090 0", "[390-390] 0 DELETE OVERLAP_UNKNOWN 02000000005453aeac4d76e7580a708a2823a4d7a4f06090 0", "[391-391] 0 DELETE OVERLAP_UNKNOWN 0200000000546a44ac4d76e7580a708a2823a4d7a4f06090 0", "[392-392] 0 DELETE OVERLAP_UNKNOWN 02000000005480dbac4d76e7580a708a2823a4d7a4f06090 0", "[393-393] 0 DELETE OVERLAP_UNKNOWN 0200000000548cb3ac4d76e7580a708a2823a4d7a4f06090 0", "[394-394] 0 DELETE OVERLAP_UNKNOWN 0200000000549a25ac4d76e7580a708a2823a4d7a4f06090 0", "[395-395] 0 DELETE OVERLAP_UNKNOWN 020000000054b359ac4d76e7580a708a2823a4d7a4f06090 0", "[396-396] 0 DELETE OVERLAP_UNKNOWN 020000000054c19dac4d76e7580a708a2823a4d7a4f06090 0", "[397-397] 0 DELETE OVERLAP_UNKNOWN 020000000054d757ac4d76e7580a708a2823a4d7a4f06090 0", ... "[1085-1085] 0 DELETE OVERLAP_UNKNOWN 02000000002a0b20bd4798638f237008ff42fbca276b52a2 0", "[1087-1506] 1 DATA NONOVERLAPPING 020000000000047e3b452de14ceaad2e78a87526026d2290 326.10 KB", "[1087-1506] 1 DATA NONOVERLAPPING 020000000000047e3b452de14ceaad2e78a87526026d2290 326.10 KB", ... "[2800-2800] 0 DELETE OVERLAP_UNKNOWN 02000000002f12d6bd4798638f237008ff42fbca276b52a2 0" Reason: The duplicate table will go through ordered data compaction. Due to the special distribution of the table, the input rowset will be cut by the ordered data compaction, resulting in the full compaction only being performed on a part of the rowsets. Solution: For full compaction, prohibit ordered data compaction. --- be/src/olap/compaction.cpp | 6 +- be/src/olap/full_compaction.cpp | 3 + ...t_full_compaction_with_ordered_data.groovy | 208 ++++++++++++++++++ 3 files changed, 215 insertions(+), 2 deletions(-) create mode 100644 regression-test/suites/fault_injection_p0/test_full_compaction_with_ordered_data.groovy diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index 738087a702f0709..d707349132036c1 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -44,6 +44,7 @@ #include "io/fs/file_system.h" #include "io/fs/file_writer.h" #include "io/fs/remote_file_system.h" +#include "io/io_common.h" #include "olap/cumulative_compaction_policy.h" #include "olap/cumulative_compaction_time_series_policy.h" #include "olap/data_dir.h" @@ -345,8 +346,9 @@ bool CompactionMixin::handle_ordered_data_compaction() { if (!config::enable_ordered_data_compaction) { return false; } - if (compaction_type() == ReaderType::READER_COLD_DATA_COMPACTION) { - // The remote file system does not support to link files. + if (compaction_type() == ReaderType::READER_COLD_DATA_COMPACTION || + compaction_type() == ReaderType::READER_FULL_COMPACTION) { + // The remote file system and full compaction does not support to link files. return false; } if (_tablet->keys_type() == KeysType::UNIQUE_KEYS && diff --git a/be/src/olap/full_compaction.cpp b/be/src/olap/full_compaction.cpp index 9d675f731924c19..529efa2e069faa1 100644 --- a/be/src/olap/full_compaction.cpp +++ b/be/src/olap/full_compaction.cpp @@ -59,6 +59,9 @@ Status FullCompaction::prepare_compact() { std::unique_lock cumu_lock(tablet()->get_cumulative_compaction_lock()); tablet()->set_is_full_compaction_running(true); + DBUG_EXECUTE_IF("FullCompaction.prepare_compact.set_cumu_point", + { tablet()->set_cumulative_layer_point(tablet()->max_version_unlocked() + 1); }) + // 1. pick rowsets to compact RETURN_IF_ERROR(pick_rowsets_to_compact()); diff --git a/regression-test/suites/fault_injection_p0/test_full_compaction_with_ordered_data.groovy b/regression-test/suites/fault_injection_p0/test_full_compaction_with_ordered_data.groovy new file mode 100644 index 000000000000000..c6dfa6b885cf6c5 --- /dev/null +++ b/regression-test/suites/fault_injection_p0/test_full_compaction_with_ordered_data.groovy @@ -0,0 +1,208 @@ +// 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. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite("test_full_compaction_with_ordered_data","nonConcurrent") { + if (isCloudMode()) { + return + } + def tableName = "test_full_compaction_with_ordered_data" + + sql """ DROP TABLE IF EXISTS ${tableName} """ + + String backend_id; + + def backendId_to_backendIP = [:] + def backendId_to_backendHttpPort = [:] + getBackendIpHttpPort(backendId_to_backendIP, backendId_to_backendHttpPort); + + backend_id = backendId_to_backendIP.keySet()[0] + + sql """ + CREATE TABLE IF NOT EXISTS ${tableName} ( + `k` int , + `v` int , + ) engine=olap + DUPLICATE KEY(k) + DISTRIBUTED BY HASH(k) + BUCKETS 3 + properties( + "replication_num" = "1", + "disable_auto_compaction" = "true") + """ + sql """ INSERT INTO ${tableName} VALUES (0,0),(1,1),(2,2)""" + sql """ delete from ${tableName} where k=0""" + sql """ delete from ${tableName} where k=1""" + sql """ delete from ${tableName} where k=2""" + + def exception = false; + try { + def tablets = sql_return_maparray """ show tablets from ${tableName}; """ + + def replicaNum = get_table_replica_num(tableName) + logger.info("get table replica num: " + replicaNum) + // before full compaction, there are 12 rowsets. + int rowsetCount = 0 + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + rowsetCount +=((List) tabletJson.rowsets).size() + } + assert (rowsetCount == 5 * replicaNum * 3) + + // trigger full compactions for all tablets in ${tableName} + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + times = 1 + + do{ + (code, out, err) = be_run_full_compaction(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + ++times + sleep(2000) + } while (parseJson(out.trim()).status.toLowerCase()!="success" && times<=10) + + } + + // wait for full compaction done + for (def tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + // after full compaction, there is only 1 rowset. + + rowsetCount = 0 + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + rowsetCount +=((List) tabletJson.rowsets).size() + } + assert (rowsetCount == 1 * replicaNum * 3) + } catch (Exception e) { + logger.info(e.getMessage()) + exception = true; + } finally { + assertFalse(exception) + } + + sql """ delete from ${tableName} where k=0""" + sql """ delete from ${tableName} where k=1""" + sql """ delete from ${tableName} where k=2""" + sql """ delete from ${tableName} where k=3""" + sql """ delete from ${tableName} where k=4""" + sql """ delete from ${tableName} where k=5""" + sql """ delete from ${tableName} where k=6""" + sql """ delete from ${tableName} where k=7""" + sql """ delete from ${tableName} where k=8""" + sql """ delete from ${tableName} where k=9""" + sql """ INSERT INTO ${tableName} VALUES (10,10)""" + + GetDebugPoint().clearDebugPointsForAllBEs() + + exception = false; + try { + GetDebugPoint().enableDebugPointForAllBEs("FullCompaction.prepare_compact.set_cumu_point") + def tablets = sql_return_maparray """ show tablets from ${tableName}; """ + + def replicaNum = get_table_replica_num(tableName) + logger.info("get table replica num: " + replicaNum) + // before full compaction, there are 12 rowsets. + int rowsetCount = 0 + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + rowsetCount +=((List) tabletJson.rowsets).size() + } + assert (rowsetCount == 12 * replicaNum * 3) + + // trigger full compactions for all tablets in ${tableName} + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + times = 1 + + do{ + (code, out, err) = be_run_full_compaction(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + ++times + sleep(2000) + } while (parseJson(out.trim()).status.toLowerCase()!="success" && times<=10) + + } + + // wait for full compaction done + for (def tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + // after full compaction, there is only 1 rowset. + + rowsetCount = 0 + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + rowsetCount +=((List) tabletJson.rowsets).size() + } + assert (rowsetCount == 1 * replicaNum * 3) + } catch (Exception e) { + logger.info(e.getMessage()) + exception = true; + } finally { + GetDebugPoint().disableDebugPointForAllBEs("FullCompaction.prepare_compact.set_cumu_point") + assertFalse(exception) + } +} From b298a7dce14dd1fb1183be852eede09518c3c84f Mon Sep 17 00:00:00 2001 From: meiyi Date: Fri, 22 Nov 2024 09:52:43 +0800 Subject: [PATCH 027/110] [fix](regression) fix unique_with_mow_c_p0/test_schema_change_ck (#44342) --- .../test_schema_change_ck.out | 248 +++++++++--------- .../test_schema_change_ck.groovy | 44 ++-- 2 files changed, 146 insertions(+), 146 deletions(-) diff --git a/regression-test/data/unique_with_mow_c_p0/test_schema_change_ck.out b/regression-test/data/unique_with_mow_c_p0/test_schema_change_ck.out index 56cba918faa01d7..a07829315f8722b 100644 --- a/regression-test/data/unique_with_mow_c_p0/test_schema_change_ck.out +++ b/regression-test/data/unique_with_mow_c_p0/test_schema_change_ck.out @@ -1,157 +1,157 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select_original -- -11 28 38 10 29 39 +11 28 38 -- !select_add_c4 -- -11 28 38 \N 10 29 39 \N -13 27 36 40 +11 28 38 \N 12 26 37 40 +13 27 36 40 -- !select_add_c5 -- -11 \N 28 38 \N 10 \N 29 39 \N -13 \N 27 36 40 +11 \N 28 38 \N 12 \N 26 37 40 -15 50 20 34 40 +13 \N 27 36 40 14 50 20 35 40 +15 50 20 34 40 -- !select_add_c6 -- -11 \N 28 \N 38 \N 10 \N 29 \N 39 \N -13 \N 27 \N 36 40 +11 \N 28 \N 38 \N 12 \N 26 \N 37 40 -15 50 20 \N 34 40 +13 \N 27 \N 36 40 14 50 20 \N 35 40 -17 50 20 60 32 40 +15 50 20 \N 34 40 16 50 20 60 33 40 +17 50 20 60 32 40 -- !select_add_k2 -- -11 \N \N 28 \N 38 \N 10 \N \N 29 \N 39 \N -13 \N \N 27 \N 36 40 +11 \N \N 28 \N 38 \N 12 \N \N 26 \N 37 40 -15 \N 50 20 \N 34 40 +13 \N \N 27 \N 36 40 14 \N 50 20 \N 35 40 -17 \N 50 20 60 32 40 +15 \N 50 20 \N 34 40 16 \N 50 20 60 33 40 -19 200 \N 20 \N 30 \N +17 \N 50 20 60 32 40 18 200 \N 20 \N 31 \N +19 200 \N 20 \N 30 \N -- !select_drop_c4 -- -11 \N \N 28 \N 38 10 \N \N 29 \N 39 -13 \N \N 27 \N 36 +11 \N \N 28 \N 38 +118 200 \N 20 \N 31 +119 200 \N 20 \N 30 12 \N \N 26 \N 37 -15 \N 50 20 \N 34 +13 \N \N 27 \N 36 14 \N 50 20 \N 35 -17 \N 50 20 60 32 +15 \N 50 20 \N 34 16 \N 50 20 60 33 -19 200 \N 20 \N 30 +17 \N 50 20 60 32 18 200 \N 20 \N 31 -119 200 \N 20 \N 30 -118 200 \N 20 \N 31 +19 200 \N 20 \N 30 -- !select_drop_c5 -- -11 \N 28 \N 38 10 \N 29 \N 39 -13 \N 27 \N 36 +11 \N 28 \N 38 +116 200 20 \N 33 +117 200 20 \N 32 +118 200 20 \N 31 +119 200 20 \N 30 12 \N 26 \N 37 -15 \N 20 \N 34 +13 \N 27 \N 36 14 \N 20 \N 35 -17 \N 20 60 32 +15 \N 20 \N 34 16 \N 20 60 33 -19 200 20 \N 30 +17 \N 20 60 32 18 200 20 \N 31 -119 200 20 \N 30 -118 200 20 \N 31 -117 200 20 \N 32 -116 200 20 \N 33 +19 200 20 \N 30 -- !select_drop_c6 -- -11 \N 28 38 10 \N 29 39 -13 \N 27 36 +11 \N 28 38 +114 200 24 35 +115 200 25 34 +116 200 20 33 +117 200 20 32 +118 200 20 31 +119 200 20 30 12 \N 26 37 -15 \N 20 34 +13 \N 27 36 14 \N 20 35 -17 \N 20 32 +15 \N 20 34 16 \N 20 33 -19 200 20 30 +17 \N 20 32 18 200 20 31 -119 200 20 30 -118 200 20 31 -117 200 20 32 -116 200 20 33 -115 200 25 34 -114 200 24 35 +19 200 20 30 -- !select_reorder -- -11 \N 38 28 10 \N 39 29 -13 \N 36 27 +11 \N 38 28 +112 200 37 22 +113 200 36 23 +114 200 35 24 +115 200 34 25 +116 200 33 20 +117 200 32 20 +118 200 31 20 +119 200 30 20 12 \N 37 26 -15 \N 34 20 +13 \N 36 27 14 \N 35 20 -17 \N 32 20 +15 \N 34 20 16 \N 33 20 -19 200 30 20 +17 \N 32 20 18 200 31 20 -119 200 30 20 -118 200 31 20 -117 200 32 20 -116 200 33 20 -115 200 34 25 -114 200 35 24 -113 200 36 23 -112 200 37 22 +19 200 30 20 -- !select_modify_k2 -- -11 \N 38 28 10 \N 39 29 -13 \N 36 27 +11 \N 38 28 +110 200 39 20 +111 200 38 21 +112 200 37 22 +113 200 36 23 +114 200 35 24 +115 200 34 25 +116 200 33 20 +117 200 32 20 +118 200 31 20 +119 200 30 20 12 \N 37 26 -15 \N 34 20 +13 \N 36 27 14 \N 35 20 -17 \N 32 20 +15 \N 34 20 16 \N 33 20 -19 200 30 20 +17 \N 32 20 18 200 31 20 -119 200 30 20 -118 200 31 20 -117 200 32 20 -116 200 33 20 -115 200 34 25 -114 200 35 24 -113 200 36 23 -112 200 37 22 -111 200 38 21 -110 200 39 20 +19 200 30 20 -- !select_create_mv_base -- -11 \N 38 28 10 \N 39 29 -13 \N 36 27 +11 \N 38 28 +110 200 39 20 +111 200 38 21 +112 200 37 22 +113 200 36 23 +114 200 35 24 +115 200 34 25 +116 200 33 20 +117 200 32 20 +118 200 31 20 +119 200 30 20 12 \N 37 26 -15 \N 34 20 +13 \N 36 27 14 \N 35 20 -17 \N 32 20 +15 \N 34 20 16 \N 33 20 -19 200 30 20 +17 \N 32 20 18 200 31 20 -119 200 30 20 -118 200 31 20 -117 200 32 20 -116 200 33 20 -115 200 34 25 -114 200 35 24 -113 200 36 23 -112 200 37 22 -111 200 38 21 -110 200 39 20 -211 200 38 21 +19 200 30 20 210 200 39 20 +211 200 38 21 -- !select_create_mv_mv -- 10 \N 29 @@ -178,30 +178,30 @@ 211 200 21 -- !select_create_rollup_base -- -11 \N 38 28 10 \N 39 29 -13 \N 36 27 +11 \N 38 28 +110 200 39 20 +111 200 38 21 +112 200 37 22 +113 200 36 23 +114 200 35 24 +115 200 34 25 +116 200 33 20 +117 200 32 20 +118 200 31 20 +119 200 30 20 12 \N 37 26 -15 \N 34 20 +13 \N 36 27 14 \N 35 20 -17 \N 32 20 +15 \N 34 20 16 \N 33 20 -19 200 30 20 +17 \N 32 20 18 200 31 20 -119 200 30 20 -118 200 31 20 -117 200 32 20 -116 200 33 20 -115 200 34 25 -114 200 35 24 -113 200 36 23 -112 200 37 22 -111 200 38 21 -110 200 39 20 -211 200 38 21 +19 200 30 20 210 200 39 20 -311 200 38 21 +211 200 38 21 310 200 39 20 +311 200 38 21 -- !select_create_rollup_roll -- \N 10 29 @@ -230,36 +230,36 @@ 200 311 21 -- !select_add_partition -- -10011 200 38 21 10010 200 39 20 +10011 200 38 21 -- !select_truncate -- -13 \N 36 27 -12 \N 37 26 -11 \N 38 28 10 \N 39 29 +11 \N 38 28 +12 \N 37 26 +13 \N 36 27 -- !select_rollup_base -- -12 22 31 41 51 11 21 32 42 52 +12 22 31 41 51 -- !select_rollup_roll -- 21 11 42 32 22 12 41 31 -- !select_rollup_base_sc -- -12 22 31 41 51 11 21 32 42 52 +12 22 31 41 51 -- !select_rollup_roll_sc -- 21 11 42 32 22 12 41 31 -- !select_rollup_base_sc1 -- -12 22 31 41 51 11 21 32 42 52 -14 24 33 43 53 +12 22 31 41 51 13 23 34 44 54 +14 24 33 43 53 -- !select_rollup_roll_sc1 -- 21 11 42 32 @@ -268,12 +268,12 @@ 24 14 43 33 -- !select_restore_base2 -- -12 22 31 41 51 11 21 32 42 52 -14 24 33 43 53 +12 22 31 41 51 13 23 34 44 54 -16 26 33 43 53 +14 24 33 43 53 15 25 34 44 54 +16 26 33 43 53 -- !select_restore_roll2 -- 21 11 42 32 @@ -284,10 +284,10 @@ 26 16 43 33 -- !select_restore_base -- -12 22 31 41 51 11 21 32 42 52 -14 24 33 43 53 +12 22 31 41 51 13 23 34 44 54 +14 24 33 43 53 -- !select_restore_roll -- 21 11 42 32 @@ -296,12 +296,12 @@ 24 14 43 33 -- !select_restore_base1 -- -12 22 31 41 51 11 21 32 42 52 -14 24 33 43 53 +12 22 31 41 51 13 23 34 44 54 -18 28 33 43 53 +14 24 33 43 53 17 27 34 44 54 +18 28 33 43 53 -- !select_restore_roll1 -- 21 11 42 32 @@ -312,10 +312,10 @@ 28 18 43 33 -- !select_restore_base2 -- -12 22 31 41 51 11 21 32 42 52 -14 24 33 43 53 +12 22 31 41 51 13 23 34 44 54 +14 24 33 43 53 -- !select_restore_roll2 -- 21 11 42 32 @@ -324,12 +324,12 @@ 24 14 43 33 -- !select_restore_base3 -- -12 22 31 41 51 11 21 32 42 52 -14 24 33 43 53 +12 22 31 41 51 13 23 34 44 54 -18 28 33 43 53 +14 24 33 43 53 17 27 34 44 54 +18 28 33 43 53 -- !select_restore_roll4 -- 21 11 42 32 diff --git a/regression-test/suites/unique_with_mow_c_p0/test_schema_change_ck.groovy b/regression-test/suites/unique_with_mow_c_p0/test_schema_change_ck.groovy index a05b2790d5f329a..09849d9fce22db3 100644 --- a/regression-test/suites/unique_with_mow_c_p0/test_schema_change_ck.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/test_schema_change_ck.groovy @@ -69,32 +69,32 @@ suite("test_schema_change_ck") { """ sql """ INSERT INTO ${tableName} VALUES (11, 28, 38), (10, 29, 39) """ - qt_select_original """select * from ${tableName}""" + order_qt_select_original """select * from ${tableName}""" /****** add value column ******/ // after cluster key sql """ alter table ${tableName} ADD column c4 int(11) after c3; """ assertTrue(getAlterTableState(), "add column should success") sql """ INSERT INTO ${tableName}(c1, c2, c3, c4) VALUES (13, 27, 36, 40), (12, 26, 37, 40) """ - qt_select_add_c4 """select * from ${tableName}""" + order_qt_select_add_c4 """select * from ${tableName}""" // before cluster key sql """ alter table ${tableName} ADD column c5 int(11) after c1; """ assertTrue(getAlterTableState(), "add column should success") sql """ INSERT INTO ${tableName}(c1, c2, c3, c4, c5) VALUES (15, 20, 34, 40, 50), (14, 20, 35, 40, 50) """ - qt_select_add_c5 """select * from ${tableName}""" + order_qt_select_add_c5 """select * from ${tableName}""" // in the middle of cluster key sql """ alter table ${tableName} ADD column c6 int(11) after c2; """ assertTrue(getAlterTableState(), "add column should success") sql """ INSERT INTO ${tableName}(c1, c2, c3, c4, c5, c6) VALUES (17, 20, 32, 40, 50, 60), (16, 20, 33, 40, 50, 60) """ - qt_select_add_c6 """select * from ${tableName}""" + order_qt_select_add_c6 """select * from ${tableName}""" /****** add key column ******/ sql """ alter table ${tableName} ADD column k2 int(11) key after c1; """ assertTrue(getAlterTableState(), "add column should success") sql """ INSERT INTO ${tableName}(c1, c2, c3, k2) VALUES (19, 20, 30, 200), (18, 20, 31, 200) """ - qt_select_add_k2 """select * from ${tableName}""" + order_qt_select_add_k2 """select * from ${tableName}""" /****** TODO add cluster key column is not supported ******/ @@ -102,17 +102,17 @@ suite("test_schema_change_ck") { sql """ alter table ${tableName} drop column c4; """ assertTrue(getAlterTableState(), "drop column should success") sql """ INSERT INTO ${tableName}(c1, c2, c3, k2) VALUES (119, 20, 30, 200), (118, 20, 31, 200) """ - qt_select_drop_c4 """select * from ${tableName}""" + order_qt_select_drop_c4 """select * from ${tableName}""" sql """ alter table ${tableName} drop column c5; """ assertTrue(getAlterTableState(), "drop column should success") sql """ INSERT INTO ${tableName}(c1, c2, c3, k2) VALUES (117, 20, 32, 200), (116, 20, 33, 200) """ - qt_select_drop_c5 """select * from ${tableName}""" + order_qt_select_drop_c5 """select * from ${tableName}""" sql """ alter table ${tableName} drop column c6; """ assertTrue(getAlterTableState(), "drop column should success") sql """ INSERT INTO ${tableName}(c1, c2, c3, k2) VALUES (115, 25, 34, 200), (114, 24, 35, 200) """ - qt_select_drop_c6 """select * from ${tableName}""" + order_qt_select_drop_c6 """select * from ${tableName}""" /****** drop key column ******/ test { @@ -130,13 +130,13 @@ suite("test_schema_change_ck") { sql """ alter table ${tableName} order by(c1, k2, c3, c2); """ assertTrue(getAlterTableState(), "reorder should success") sql """ INSERT INTO ${tableName}(c1, c2, c3, k2) VALUES (113, 23, 36, 200), (112, 22, 37, 200) """ - qt_select_reorder """select * from ${tableName}""" + order_qt_select_reorder """select * from ${tableName}""" /****** modify key column data type ******/ sql """ alter table ${tableName} modify column k2 BIGINT key; """ assertTrue(getAlterTableState(), "modify should success") sql """ INSERT INTO ${tableName}(c1, c2, c3, k2) VALUES (111, 21, 38, 200), (110, 20, 39, 200) """ - qt_select_modify_k2 """select * from ${tableName}""" + order_qt_select_modify_k2 """select * from ${tableName}""" /****** TODO does not support modify cluster key column data type ******/ test { @@ -149,7 +149,7 @@ suite("test_schema_change_ck") { sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name}""" createMV """ create materialized view ${mv_name} as select c1, k2, c2 from ${tableName}; """ sql """ INSERT INTO ${tableName}(c1, c2, c3, k2) VALUES (211, 21, 38, 200), (210, 20, 39, 200) """ - qt_select_create_mv_base """select * from ${tableName}""" + order_qt_select_create_mv_base """select * from ${tableName}""" /*Awaitility.await().atMost(100, SECONDS).pollInterval(4, SECONDS).until( { def result = sql """explain select c1, c3 from ${tableName}""" @@ -165,7 +165,7 @@ suite("test_schema_change_ck") { time 600 } sql """ INSERT INTO ${tableName}(c1, c2, c3, k2) VALUES (311, 21, 38, 200), (310, 20, 39, 200) """ - qt_select_create_rollup_base """select * from ${tableName}""" + order_qt_select_create_rollup_base """select * from ${tableName}""" order_qt_select_create_rollup_roll """select k2, c1, c2 from ${tableName}""" /****** add partition ******/ @@ -180,14 +180,14 @@ suite("test_schema_change_ck") { assertEquals(partitions.size(), 2) } sql """ INSERT INTO ${tableName}(c1, c2, c3, k2) VALUES (10011, 21, 38, 200), (10010, 20, 39, 200) """ - qt_select_add_partition """select * from ${tableName} partition (p_20000)""" + order_qt_select_add_partition """select * from ${tableName} partition (p_20000)""" /****** one sql contain multi column changes ******/ /****** truncate table ******/ sql """ TRUNCATE TABLE ${tableName} """ sql """ INSERT INTO ${tableName}(c1, c2, c3) VALUES (11, 28, 38), (10, 29, 39), (12, 26, 37), (13, 27, 36) """ - qt_select_truncate """select * from ${tableName}""" + order_qt_select_truncate """select * from ${tableName}""" /****** create table with rollup ******/ tableName = tableName + "_rollup" @@ -211,16 +211,16 @@ suite("test_schema_change_ck") { ); """ sql """ INSERT INTO ${tableName} VALUES (11, 21, 32, 42, 52), (12, 22, 31, 41, 51); """ - qt_select_rollup_base """select * from ${tableName};""" + order_qt_select_rollup_base """select * from ${tableName};""" order_qt_select_rollup_roll """select k2, k1, c4, c3 from ${tableName};""" /****** specify index, not base index ******/ sql """ ALTER TABLE ${tableName} ORDER BY(k2, k1, c3, c4) from r1; """ assertTrue(getAlterTableState(), "reorder rollup should success") - qt_select_rollup_base_sc """select * from ${tableName};""" + order_qt_select_rollup_base_sc """select * from ${tableName};""" order_qt_select_rollup_roll_sc """select k2, k1, c4, c3 from ${tableName};""" sql """ INSERT INTO ${tableName} VALUES (13, 23, 34, 44, 54), (14, 24, 33, 43, 53); """ - qt_select_rollup_base_sc1 """select * from ${tableName};""" + order_qt_select_rollup_base_sc1 """select * from ${tableName};""" order_qt_select_rollup_roll_sc1 """select k2, k1, c4, c3 from ${tableName};""" /****** backup restore ******/ @@ -238,7 +238,7 @@ suite("test_schema_change_ck") { def snapshot = syncer.getSnapshotTimestamp(repoName, backup) assertTrue(snapshot != null) sql """ INSERT INTO ${tableName} VALUES (15, 25, 34, 44, 54), (16, 26, 33, 43, 53); """ - qt_select_restore_base2 """select * from ${tableName};""" + order_qt_select_restore_base2 """select * from ${tableName};""" order_qt_select_restore_roll2 """select k2, k1, c4, c3 from ${tableName};""" // restore @@ -247,10 +247,10 @@ suite("test_schema_change_ck") { syncer.waitAllRestoreFinish(context.dbName) result = sql """ show tablets from ${tableName}; """ logger.info("tablets 1: ${result}") - qt_select_restore_base """select * from ${tableName};""" + order_qt_select_restore_base """select * from ${tableName};""" order_qt_select_restore_roll """select k2, k1, c4, c3 from ${tableName};""" sql """ INSERT INTO ${tableName} VALUES (17, 27, 34, 44, 54), (18, 28, 33, 43, 53); """ - qt_select_restore_base1 """select * from ${tableName};""" + order_qt_select_restore_base1 """select * from ${tableName};""" order_qt_select_restore_roll1 """select k2, k1, c4, c3 from ${tableName};""" // restore @@ -259,10 +259,10 @@ suite("test_schema_change_ck") { syncer.waitAllRestoreFinish(context.dbName) result = sql """ show tablets from ${tableName}; """ logger.info("tablets 2: ${result}") - qt_select_restore_base2 """select * from ${tableName};""" + order_qt_select_restore_base2 """select * from ${tableName};""" order_qt_select_restore_roll2 """select k2, k1, c4, c3 from ${tableName};""" sql """ INSERT INTO ${tableName} VALUES (17, 27, 34, 44, 54), (18, 28, 33, 43, 53); """ - qt_select_restore_base3 """select * from ${tableName};""" + order_qt_select_restore_base3 """select * from ${tableName};""" order_qt_select_restore_roll4 """select k2, k1, c4, c3 from ${tableName};""" sql "DROP REPOSITORY `${repoName}`" From 074bd7c1f012f98468d89421951a7c0904efd036 Mon Sep 17 00:00:00 2001 From: qiye Date: Fri, 22 Nov 2024 10:13:54 +0800 Subject: [PATCH 028/110] [fix](bloom filter)Fix drop column with bloom filter index (#44361) Problem Summary: 1. When drop column with bloom filter, we modify the bloom filter column info 2. When replay editLog, we rebuild bloom filter info by table schema. Related PR: #41369 ### Release note Fix drop column with bloom filter index --- .../doris/alter/SchemaChangeHandler.java | 24 +++++++++++++++- .../test_bloom_filter_drop_column.out | 2 +- .../test_bloom_filter_drop_column.groovy | 28 ++++++++++++++----- 3 files changed, 45 insertions(+), 9 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index b7a0fa5cfbc7462..6eaf7d5522c96d2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -434,9 +434,12 @@ private boolean processDropColumn(DropColumnClause alterClause, OlapTable olapTa // drop bloom filter column Set bfCols = olapTable.getCopiedBfColumns(); if (bfCols != null) { - Set newBfCols = new HashSet<>(); + Set newBfCols = null; for (String bfCol : bfCols) { if (!bfCol.equalsIgnoreCase(dropColName)) { + if (newBfCols == null) { + newBfCols = Sets.newHashSet(); + } newBfCols.add(bfCol); } } @@ -2953,6 +2956,25 @@ public void modifyTableLightSchemaChange(String rawSql, Database db, OlapTable o LOG.info("finished modify table's add or drop or modify columns. table: {}, job: {}, is replay: {}", olapTable.getName(), jobId, isReplay); } + // for bloom filter, rebuild bloom filter info by table schema in replay + if (isReplay) { + Set bfCols = olapTable.getCopiedBfColumns(); + if (bfCols != null) { + List columns = olapTable.getBaseSchema(); + Set newBfCols = null; + for (String bfCol : bfCols) { + for (Column column : columns) { + if (column.getName().equalsIgnoreCase(bfCol)) { + if (newBfCols == null) { + newBfCols = Sets.newHashSet(); + } + newBfCols.add(column.getName()); + } + } + } + olapTable.setBloomFilterInfo(newBfCols, olapTable.getBfFpp()); + } + } } public void replayModifyTableLightSchemaChange(TableAddOrDropColumnsInfo info) diff --git a/regression-test/data/bloom_filter_p0/test_bloom_filter_drop_column.out b/regression-test/data/bloom_filter_p0/test_bloom_filter_drop_column.out index 2c6ca8d224b7282..14334dfb4b5c48e 100644 --- a/regression-test/data/bloom_filter_p0/test_bloom_filter_drop_column.out +++ b/regression-test/data/bloom_filter_p0/test_bloom_filter_drop_column.out @@ -1,6 +1,6 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -1 1 +1 1 1 -- !select -- 1 \N diff --git a/regression-test/suites/bloom_filter_p0/test_bloom_filter_drop_column.groovy b/regression-test/suites/bloom_filter_p0/test_bloom_filter_drop_column.groovy index f426d4fca10a79a..d83c70af30c709c 100644 --- a/regression-test/suites/bloom_filter_p0/test_bloom_filter_drop_column.groovy +++ b/regression-test/suites/bloom_filter_p0/test_bloom_filter_drop_column.groovy @@ -21,13 +21,14 @@ suite("test_bloom_filter_drop_column") { sql """CREATE TABLE IF NOT EXISTS ${table_name} ( `a` varchar(150) NULL, - `c1` varchar(10) + `c1` varchar(10), + `c2` varchar(10) ) ENGINE=OLAP DUPLICATE KEY(`a`) DISTRIBUTED BY HASH(`a`) BUCKETS 1 PROPERTIES ( "replication_allocation" = "tag.location.default: 1", - "bloom_filter_columns" = "c1", + "bloom_filter_columns" = "c1, c2", "in_memory" = "false", "storage_format" = "V2" )""" @@ -51,12 +52,12 @@ suite("test_bloom_filter_drop_column") { assertTrue(useTime <= OpTimeout, "wait_for_latest_op_on_table_finish timeout") } - def assertShowCreateTableWithRetry = { tableName, expectedCondition, maxRetries, waitSeconds -> + def assertShowCreateTableWithRetry = { tableName, expectedCondition, contains, maxRetries, waitSeconds -> int attempt = 0 while (attempt < maxRetries) { def res = sql """SHOW CREATE TABLE ${tableName}""" log.info("Attempt ${attempt + 1}: show table: ${res}") - if (res && res.size() > 0 && res[0][1].contains(expectedCondition)) { + if (res && res.size() > 0 && ((contains && res[0][1].contains(expectedCondition)) || (!contains && !res[0][1].contains(expectedCondition)))) { logger.info("Attempt ${attempt + 1}: Condition met.") return } else { @@ -70,21 +71,34 @@ suite("test_bloom_filter_drop_column") { def finalRes = sql """SHOW CREATE TABLE ${tableName}""" log.info("Final attempt: show table: ${finalRes}") assertTrue(finalRes && finalRes.size() > 0, "SHOW CREATE TABLE return empty or null") - assertTrue(finalRes[0][1].contains(expectedCondition), "expected\"${expectedCondition}\",actural: ${finalRes[0][1]}") + if (contains) { + assertTrue(finalRes[0][1].contains(expectedCondition), "expected to contain \"${expectedCondition}\", actual: ${finalRes[0][1]}") + } else { + assertTrue(!finalRes[0][1].contains(expectedCondition), "expected not to contain \"${expectedCondition}\", actual: ${finalRes[0][1]}") + } } - sql """INSERT INTO ${table_name} values ('1', '1')""" + sql """INSERT INTO ${table_name} values ('1', '1', '1')""" sql "sync" qt_select """select * from ${table_name} order by a""" + assertShowCreateTableWithRetry(table_name, "\"bloom_filter_columns\" = \"c1, c2\"", true, 3, 30) // drop column c1 sql """ALTER TABLE ${table_name} DROP COLUMN c1""" wait_for_latest_op_on_table_finish(table_name, timeout) sql "sync" // show create table with retry logic - assertShowCreateTableWithRetry(table_name, "\"bloom_filter_columns\" = \"\"", 3, 30) + assertShowCreateTableWithRetry(table_name, "\"bloom_filter_columns\" = \"c2\"", true, 3, 30) + + // drop column c2 + sql """ALTER TABLE ${table_name} DROP COLUMN c2""" + wait_for_latest_op_on_table_finish(table_name, timeout) + sql "sync" + + // show create table with retry logic + assertShowCreateTableWithRetry(table_name, "\"bloom_filter_columns\" = \"\"", false, 3, 30) // add new column c1 sql """ALTER TABLE ${table_name} ADD COLUMN c1 ARRAY""" From 14711d7ca9577dc6f56b16d8597e08235269423a Mon Sep 17 00:00:00 2001 From: Socrates Date: Fri, 22 Nov 2024 10:21:09 +0800 Subject: [PATCH 029/110] [enhance](orc) Optimize ORC Predicate Pushdown for OR-connected Predicate (#43255) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What problem does this PR solve? Problem Summary: This issue addresses a limitation in Apache Doris where only predicates joined by AND are pushed down to the ORC reader, leaving OR-connected predicates unoptimized. By extending pushdown functionality to handle these OR conditions, the aim is to better leverage ORC’s predicate pushdown capabilities, reducing data reads and improving query performance. --- be/src/runtime/exec_env.h | 1 + be/src/vec/exec/format/orc/vorc_reader.cpp | 478 ++++++++++++------ be/src/vec/exec/format/orc/vorc_reader.h | 37 +- be/test/exec/test_data/orc_scanner/orders.orc | Bin 0 -> 1293 bytes be/test/testutil/desc_tbl_builder.cpp | 29 +- be/test/testutil/desc_tbl_builder.h | 17 +- be/test/vec/exec/orc_reader_test.cpp | 155 ++++++ .../external_table_p0/hive/test_hive_orc.out | 48 ++ .../hive/test_hive_orc.groovy | 12 + 9 files changed, 595 insertions(+), 182 deletions(-) create mode 100644 be/test/exec/test_data/orc_scanner/orders.orc create mode 100644 be/test/vec/exec/orc_reader_test.cpp diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 813f0833ad40615..a27936f5f0d88b9 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -294,6 +294,7 @@ class ExecEnv { static void set_tracking_memory(bool tracking_memory) { _s_tracking_memory.store(tracking_memory, std::memory_order_release); } + void set_orc_memory_pool(orc::MemoryPool* pool) { _orc_memory_pool = pool; } #endif LoadStreamMapPool* load_stream_map_pool() { return _load_stream_map_pool.get(); } diff --git a/be/src/vec/exec/format/orc/vorc_reader.cpp b/be/src/vec/exec/format/orc/vorc_reader.cpp index 5d6ae4024c75610..70f3f6f003f6117 100644 --- a/be/src/vec/exec/format/orc/vorc_reader.cpp +++ b/be/src/vec/exec/format/orc/vorc_reader.cpp @@ -18,13 +18,14 @@ #include "vorc_reader.h" #include -#include #include +#include #include #include #include #include +#include // IWYU pragma: no_include #include // IWYU pragma: keep #include @@ -33,12 +34,10 @@ #include #include #include -#include #include "cctz/civil_time.h" #include "cctz/time_zone.h" #include "common/exception.h" -#include "exec/olap_utils.h" #include "exprs/create_predicate_function.h" #include "exprs/hybrid_set.h" #include "gutil/strings/substitute.h" @@ -55,6 +54,7 @@ #include "runtime/descriptors.h" #include "runtime/primitive_type.h" #include "runtime/thread_context.h" +#include "util/runtime_profile.h" #include "util/slice.h" #include "util/timezone_utils.h" #include "vec/columns/column.h" @@ -71,15 +71,13 @@ #include "vec/data_types/data_type_map.h" #include "vec/data_types/data_type_nullable.h" #include "vec/data_types/data_type_struct.h" -#include "vec/exec/format/orc/orc_memory_pool.h" #include "vec/exec/format/table/transactional_hive_common.h" #include "vec/exprs/vbloom_predicate.h" #include "vec/exprs/vdirect_in_predicate.h" #include "vec/exprs/vectorized_fn_call.h" +#include "vec/exprs/vexpr_fwd.h" #include "vec/exprs/vin_predicate.h" -#include "vec/exprs/vliteral.h" #include "vec/exprs/vruntimefilter_wrapper.h" -#include "vec/exprs/vslot_ref.h" #include "vec/runtime/vdatetime_value.h" namespace doris { @@ -237,6 +235,10 @@ void OrcReader::_init_profile() { ADD_CHILD_TIMER_WITH_LEVEL(_profile, "DecodeNullMapTime", orc_profile, 1); _orc_profile.filter_block_time = ADD_CHILD_TIMER_WITH_LEVEL(_profile, "FilterBlockTime", orc_profile, 1); + _orc_profile.selected_row_group_count = + ADD_COUNTER_WITH_LEVEL(_profile, "SelectedRowGroupCount", TUnit::UNIT, 1); + _orc_profile.evaluated_row_group_count = + ADD_COUNTER_WITH_LEVEL(_profile, "EvaluatedRowGroupCount", TUnit::UNIT, 1); } } @@ -259,6 +261,7 @@ Status OrcReader::_create_file_reader() { try { orc::ReaderOptions options; options.setMemoryPool(*ExecEnv::GetInstance()->orc_memory_pool()); + options.setReaderMetrics(&_reader_metrics); _reader = orc::createReader( std::unique_ptr(_file_input_stream.release()), options); } catch (std::exception& e) { @@ -388,6 +391,9 @@ Status OrcReader::_init_read_columns() { } _col_name_to_file_col_name[col_name] = read_col; + // TODO: refactor this + std::transform(read_col.begin(), read_col.end(), read_col.begin(), ::tolower); + _col_name_to_file_col_name_low_case[col_name] = read_col; } } return Status::OK(); @@ -461,8 +467,10 @@ static std::unordered_map TYPEKIND_TO_PRE {orc::TypeKind::BOOLEAN, orc::PredicateDataType::BOOLEAN}}; template -std::tuple convert_to_orc_literal(const orc::Type* type, const void* value, - int precision, int scale) { +std::tuple convert_to_orc_literal(const orc::Type* type, + StringRef& literal_data, int precision, + int scale) { + const auto* value = literal_data.data; try { switch (type->getKind()) { case orc::TypeKind::BOOLEAN: @@ -486,8 +494,7 @@ std::tuple convert_to_orc_literal(const orc::Type* type, con case orc::TypeKind::CHAR: [[fallthrough]]; case orc::TypeKind::VARCHAR: { - StringRef* string_value = (StringRef*)value; - return std::make_tuple(true, orc::Literal(string_value->data, string_value->size)); + return std::make_tuple(true, orc::Literal(literal_data.data, literal_data.size)); } case orc::TypeKind::DECIMAL: { int128_t decimal_value; @@ -558,190 +565,353 @@ std::tuple convert_to_orc_literal(const orc::Type* type, con } } -template -std::vector value_range_to_predicate( - const ColumnValueRange& col_val_range, const orc::Type* type, - std::vector* unsupported_pushdown_types) { - std::vector predicates; - - PrimitiveType src_type = OrcReader::convert_to_doris_type(type).type; - if (src_type != primitive_type) { - if (!(is_string_type(src_type) && is_string_type(primitive_type))) { - // not support schema change - return predicates; - } +std::tuple OrcReader::_make_orc_literal( + const VSlotRef* slot_ref, const VLiteral* literal) { + auto file_col_name_low_case = _col_name_to_file_col_name_low_case[slot_ref->expr_name()]; + if (!_type_map.contains(file_col_name_low_case)) { + // TODO: this is for acid table + LOG(WARNING) << "Column " << slot_ref->expr_name() << " not found in _type_map"; + return std::make_tuple(false, orc::Literal(false), orc::PredicateDataType::LONG); + } + const auto* orc_type = _type_map[file_col_name_low_case]; + if (!TYPEKIND_TO_PREDICATE_TYPE.contains(orc_type->getKind())) { + LOG(WARNING) << "Unsupported Push Down Orc Type [TypeKind=" << orc_type->getKind() << "]"; + return std::make_tuple(false, orc::Literal(false), orc::PredicateDataType::LONG); + } + const auto predicate_type = TYPEKIND_TO_PREDICATE_TYPE[orc_type->getKind()]; + if (literal == nullptr) { + // only get the predicate_type + return std::make_tuple(true, orc::Literal(true), predicate_type); + } + auto literal_data = literal->get_column_ptr()->get_data_at(0); + auto* slot = _tuple_descriptor->slots()[slot_ref->column_id()]; + auto slot_type = slot->type(); + switch (slot_type.type) { +#define M(NAME) \ + case TYPE_##NAME: { \ + auto [valid, orc_literal] = convert_to_orc_literal( \ + orc_type, literal_data, slot_type.precision, slot_type.scale); \ + return std::make_tuple(valid, orc_literal, predicate_type); \ + } +#define APPLY_FOR_PRIMITIVE_TYPE(M) \ + M(TINYINT) \ + M(SMALLINT) \ + M(INT) \ + M(BIGINT) \ + M(LARGEINT) \ + M(CHAR) \ + M(DATE) \ + M(DATETIME) \ + M(DATEV2) \ + M(DATETIMEV2) \ + M(VARCHAR) \ + M(STRING) \ + M(HLL) \ + M(DECIMAL32) \ + M(DECIMAL64) \ + M(DECIMAL128I) \ + M(DECIMAL256) \ + M(DECIMALV2) \ + M(BOOLEAN) \ + M(IPV4) \ + M(IPV6) + APPLY_FOR_PRIMITIVE_TYPE(M) +#undef M + default: { + VLOG_CRITICAL << "Unsupported Convert Orc Literal [ColName=" << slot->col_name() << "]"; + return std::make_tuple(false, orc::Literal(false), predicate_type); + } } +} - if (unsupported_pushdown_types != nullptr) { - for (vector::iterator it = unsupported_pushdown_types->begin(); - it != unsupported_pushdown_types->end(); ++it) { - if (*it == type->getKind()) { - // Unsupported type - return predicates; - } - } +// check if the slot of expr can be pushed down to orc reader +bool OrcReader::_check_slot_can_push_down(const VExprSPtr& expr) { + if (!expr->children()[0]->is_slot_ref()) { + return false; } + const auto* slot_ref = static_cast(expr->children()[0].get()); + // check if the slot exists in orc file and not partition column + return _col_name_to_file_col_name.contains(slot_ref->expr_name()) && + !_lazy_read_ctx.predicate_partition_columns.contains(slot_ref->expr_name()); +} - orc::PredicateDataType predicate_data_type; - auto type_it = TYPEKIND_TO_PREDICATE_TYPE.find(type->getKind()); - if (type_it == TYPEKIND_TO_PREDICATE_TYPE.end()) { - // Unsupported type - return predicates; - } else { - predicate_data_type = type_it->second; +// check if there are rest children of expr can be pushed down to orc reader +bool OrcReader::_check_rest_children_can_push_down(const VExprSPtr& expr) { + if (expr->children().size() < 2) { + return false; } - if (col_val_range.is_fixed_value_range()) { - OrcPredicate in_predicate; - in_predicate.col_name = col_val_range.column_name(); - in_predicate.data_type = predicate_data_type; - in_predicate.op = SQLFilterOp::FILTER_IN; - for (const auto& value : col_val_range.get_fixed_value_set()) { - auto [valid, literal] = convert_to_orc_literal( - type, &value, col_val_range.precision(), col_val_range.scale()); - if (valid) { - in_predicate.literals.push_back(literal); - } + for (size_t i = 1; i < expr->children().size(); ++i) { + if (!expr->children()[i]->is_literal()) { + return false; } - if (!in_predicate.literals.empty()) { - predicates.emplace_back(in_predicate); + } + return true; +} + +// check if the expr can be pushed down to orc reader +bool OrcReader::_check_expr_can_push_down(const VExprSPtr& expr) { + DCHECK(expr != nullptr); + switch (expr->op()) { + case TExprOpcode::COMPOUND_AND: + // at least one child can be pushed down + return std::ranges::any_of(expr->children(), [this](const auto& child) { + return _check_expr_can_push_down(child); + }); + case TExprOpcode::COMPOUND_OR: + // all children must be pushed down + return std::ranges::all_of(expr->children(), [this](const auto& child) { + return _check_expr_can_push_down(child); + }); + case TExprOpcode::COMPOUND_NOT: + DCHECK_EQ(expr->children().size(), 1); + return _check_expr_can_push_down(expr->children()[0]); + + case TExprOpcode::GE: + case TExprOpcode::GT: + case TExprOpcode::LE: + case TExprOpcode::LT: + case TExprOpcode::EQ: + case TExprOpcode::NE: + case TExprOpcode::FILTER_IN: + case TExprOpcode::FILTER_NOT_IN: + return _check_slot_can_push_down(expr) && _check_rest_children_can_push_down(expr); + + case TExprOpcode::INVALID_OPCODE: + if (expr->node_type() == TExprNodeType::FUNCTION_CALL) { + auto fn_name = expr->fn().name.function_name; + // only support is_null_pred and is_not_null_pred + if (fn_name == "is_null_pred" || fn_name == "is_not_null_pred") { + return _check_slot_can_push_down(expr); + } + VLOG_CRITICAL << "Unsupported function [funciton=" << fn_name << "]"; } - return predicates; + return false; + default: + VLOG_CRITICAL << "Unsupported Opcode [OpCode=" << expr->op() << "]"; + return false; + } +} + +bool OrcReader::_build_less_than(const VExprSPtr& expr, + std::unique_ptr& builder) { + DCHECK(expr->children().size() == 2); + DCHECK(expr->children()[0]->is_slot_ref()); + DCHECK(expr->children()[1]->is_literal()); + const auto* slot_ref = static_cast(expr->children()[0].get()); + const auto* literal = static_cast(expr->children()[1].get()); + auto [valid, orc_literal, predicate_type] = _make_orc_literal(slot_ref, literal); + if (!valid) { + return false; } + builder->lessThan(slot_ref->expr_name(), predicate_type, orc_literal); + return true; +} - const auto& high_value = col_val_range.get_range_max_value(); - const auto& low_value = col_val_range.get_range_min_value(); - const auto& high_op = col_val_range.get_range_high_op(); - const auto& low_op = col_val_range.get_range_low_op(); +bool OrcReader::_build_less_than_equals(const VExprSPtr& expr, + std::unique_ptr& builder) { + DCHECK(expr->children().size() == 2); + DCHECK(expr->children()[0]->is_slot_ref()); + DCHECK(expr->children()[1]->is_literal()); + const auto* slot_ref = static_cast(expr->children()[0].get()); + const auto* literal = static_cast(expr->children()[1].get()); + auto [valid, orc_literal, predicate_type] = _make_orc_literal(slot_ref, literal); + if (!valid) { + return false; + } + builder->lessThanEquals(slot_ref->expr_name(), predicate_type, orc_literal); + return true; +} - // orc can only push down is_null. When col_value_range._contain_null = true, only indicating that - // value can be null, not equals null, so ignore _contain_null in col_value_range - if (col_val_range.is_high_value_maximum() && high_op == SQLFilterOp::FILTER_LESS_OR_EQUAL && - col_val_range.is_low_value_mininum() && low_op == SQLFilterOp::FILTER_LARGER_OR_EQUAL) { - return predicates; +bool OrcReader::_build_equals(const VExprSPtr& expr, + std::unique_ptr& builder) { + DCHECK(expr->children().size() == 2); + DCHECK(expr->children()[0]->is_slot_ref()); + DCHECK(expr->children()[1]->is_literal()); + const auto* slot_ref = static_cast(expr->children()[0].get()); + const auto* literal = static_cast(expr->children()[1].get()); + auto [valid, orc_literal, predicate_type] = _make_orc_literal(slot_ref, literal); + if (!valid) { + return false; } + builder->equals(slot_ref->expr_name(), predicate_type, orc_literal); + return true; +} - if (low_value < high_value) { - if (!col_val_range.is_low_value_mininum() || - SQLFilterOp::FILTER_LARGER_OR_EQUAL != low_op) { - auto [valid, low_literal] = convert_to_orc_literal( - type, &low_value, col_val_range.precision(), col_val_range.scale()); - if (valid) { - OrcPredicate low_predicate; - low_predicate.col_name = col_val_range.column_name(); - low_predicate.data_type = predicate_data_type; - low_predicate.op = low_op; - low_predicate.literals.emplace_back(low_literal); - predicates.emplace_back(low_predicate); - } - } - if (!col_val_range.is_high_value_maximum() || - SQLFilterOp::FILTER_LESS_OR_EQUAL != high_op) { - auto [valid, high_literal] = convert_to_orc_literal( - type, &high_value, col_val_range.precision(), col_val_range.scale()); - if (valid) { - OrcPredicate high_predicate; - high_predicate.col_name = col_val_range.column_name(); - high_predicate.data_type = predicate_data_type; - high_predicate.op = high_op; - high_predicate.literals.emplace_back(high_literal); - predicates.emplace_back(high_predicate); - } +bool OrcReader::_build_filter_in(const VExprSPtr& expr, + std::unique_ptr& builder) { + DCHECK(expr->children().size() >= 2); + DCHECK(expr->children()[0]->is_slot_ref()); + const auto* slot_ref = static_cast(expr->children()[0].get()); + std::vector literals; + orc::PredicateDataType predicate_type = orc::PredicateDataType::LONG; + for (size_t i = 1; i < expr->children().size(); ++i) { + DCHECK(expr->children()[i]->is_literal()); + const auto* literal = static_cast(expr->children()[i].get()); + auto [valid, orc_literal, type] = _make_orc_literal(slot_ref, literal); + if (!valid) { + return false; } + literals.emplace_back(orc_literal); + predicate_type = type; } - return predicates; + DCHECK(!literals.empty()); + builder->in(slot_ref->expr_name(), predicate_type, literals); + return true; } -bool static build_search_argument(std::vector& predicates, int index, - std::unique_ptr& builder) { - if (index >= predicates.size()) { +bool OrcReader::_build_is_null(const VExprSPtr& expr, + std::unique_ptr& builder) { + DCHECK(expr->children().size() == 1); + DCHECK(expr->children()[0]->is_slot_ref()); + const auto* slot_ref = static_cast(expr->children()[0].get()); + auto [valid, _, predicate_type] = _make_orc_literal(slot_ref, nullptr); + builder->isNull(slot_ref->expr_name(), predicate_type); + return true; +} + +bool OrcReader::_build_search_argument(const VExprSPtr& expr, + std::unique_ptr& builder) { + if (expr == nullptr) { return false; } - if (index < predicates.size() - 1) { - builder->startAnd(); + + // if expr can not be pushed down, skip it and continue to next expr + if (!_check_expr_can_push_down(expr)) { + return false; } - OrcPredicate& predicate = predicates[index]; - switch (predicate.op) { - case SQLFilterOp::FILTER_IN: { - if (predicate.literals.size() == 1) { - builder->equals(predicate.col_name, predicate.data_type, predicate.literals[0]); - } else { - builder->in(predicate.col_name, predicate.data_type, predicate.literals); + + switch (expr->op()) { + case TExprOpcode::COMPOUND_AND: { + bool at_least_one_can_push_down = false; + builder->startAnd(); + for (const auto& child : expr->children()) { + if (_build_search_argument(child, builder)) { + at_least_one_can_push_down = true; + } + } + if (!at_least_one_can_push_down) { + // if all exprs can not be pushed down, builder->end() will throw exception + return false; } + builder->end(); break; } - case SQLFilterOp::FILTER_LESS: - builder->lessThan(predicate.col_name, predicate.data_type, predicate.literals[0]); + case TExprOpcode::COMPOUND_OR: + builder->startOr(); + for (const auto& child : expr->children()) { + if (!_build_search_argument(child, builder)) { + return false; + } + } + builder->end(); + break; + case TExprOpcode::COMPOUND_NOT: + builder->startNot(); + DCHECK_EQ(expr->children().size(), 1); + if (!_build_search_argument(expr->children()[0], builder)) { + return false; + } + builder->end(); break; - case SQLFilterOp::FILTER_LESS_OR_EQUAL: - builder->lessThanEquals(predicate.col_name, predicate.data_type, predicate.literals[0]); + case TExprOpcode::GE: + builder->startNot(); + if (!_build_less_than(expr, builder)) { + return false; + } + builder->end(); break; - case SQLFilterOp::FILTER_LARGER: { + case TExprOpcode::GT: builder->startNot(); - builder->lessThanEquals(predicate.col_name, predicate.data_type, predicate.literals[0]); + if (!_build_less_than_equals(expr, builder)) { + return false; + } builder->end(); break; - } - case SQLFilterOp::FILTER_LARGER_OR_EQUAL: { + case TExprOpcode::LE: + if (!_build_less_than_equals(expr, builder)) { + return false; + } + break; + case TExprOpcode::LT: + if (!_build_less_than(expr, builder)) { + return false; + } + break; + case TExprOpcode::EQ: + if (!_build_equals(expr, builder)) { + return false; + } + break; + case TExprOpcode::NE: builder->startNot(); - builder->lessThan(predicate.col_name, predicate.data_type, predicate.literals[0]); + if (!_build_equals(expr, builder)) { + return false; + } builder->end(); break; - } - default: - return false; - } - if (index < predicates.size() - 1) { - bool can_build = build_search_argument(predicates, index + 1, builder); - if (!can_build) { + case TExprOpcode::FILTER_IN: + if (!_build_filter_in(expr, builder)) { + return false; + } + break; + case TExprOpcode::FILTER_NOT_IN: + builder->startNot(); + if (!_build_filter_in(expr, builder)) { return false; } builder->end(); + break; + // is null and is not null is represented as function call + case TExprOpcode::INVALID_OPCODE: { + DCHECK(expr->node_type() == TExprNodeType::FUNCTION_CALL); + if (expr->fn().name.function_name == "is_null_pred") { + if (!_build_is_null(expr, builder)) { + return false; + } + } else if (expr->fn().name.function_name == "is_not_null_pred") { + builder->startNot(); + if (!_build_is_null(expr, builder)) { + return false; + } + builder->end(); + } else { + __builtin_unreachable(); + } + break; + } + default: { + // should not reach here, because _check_expr_can_push_down has already checked + __builtin_unreachable(); + } } return true; } -bool OrcReader::_init_search_argument( - std::unordered_map* colname_to_value_range) { - if ((!_enable_filter_by_min_max) || colname_to_value_range->empty()) { +bool OrcReader::_init_search_argument(const VExprContextSPtrs& conjuncts) { + if (!_enable_filter_by_min_max) { return false; } - std::vector predicates; - auto& root_type = _reader->getType(); - std::unordered_map type_map; - for (int i = 0; i < root_type.getSubtypeCount(); ++i) { - type_map.emplace(get_field_name_lower_case(&root_type, i), root_type.getSubtype(i)); - } - for (auto& col_name : _lazy_read_ctx.all_read_columns) { - auto iter = colname_to_value_range->find(col_name); - if (iter == colname_to_value_range->end()) { - continue; - } - auto type_it = type_map.find(_col_name_to_file_col_name[col_name]); - if (type_it == type_map.end()) { - continue; + + // build search argument, if any expr can not be pushed down, return false + auto builder = orc::SearchArgumentFactory::newBuilder(); + bool at_least_one_can_push_down = false; + builder->startAnd(); + for (const auto& expr_ctx : conjuncts) { + if (_build_search_argument(expr_ctx->root(), builder)) { + at_least_one_can_push_down = true; } - std::visit( - [&](auto& range) { - std::vector value_predicates = value_range_to_predicate( - range, type_it->second, _unsupported_pushdown_types); - for (auto& range_predicate : value_predicates) { - predicates.emplace_back(range_predicate); - } - }, - iter->second); } - if (predicates.empty()) { - return false; - } - std::unique_ptr builder = orc::SearchArgumentFactory::newBuilder(); - if (build_search_argument(predicates, 0, builder)) { - std::unique_ptr sargs = builder->build(); - _row_reader_options.searchArgument(std::move(sargs)); - return true; - } else { + if (!at_least_one_can_push_down) { + // if all exprs can not be pushed down, builder->end() will throw exception return false; } + builder->end(); + + auto sargs = builder->build(); + _profile->add_info_string("OrcReader SearchArgument: ", sargs->toString()); + _row_reader_options.searchArgument(std::move(sargs)); + return true; } Status OrcReader::set_fill_columns( @@ -854,7 +1024,7 @@ Status OrcReader::set_fill_columns( _lazy_read_ctx.can_lazy_read = true; } - if (_colname_to_value_range == nullptr || !_init_search_argument(_colname_to_value_range)) { + if (_lazy_read_ctx.conjuncts.empty() || !_init_search_argument(_lazy_read_ctx.conjuncts)) { _lazy_read_ctx.can_lazy_read = false; } try { @@ -1619,6 +1789,12 @@ std::string OrcReader::get_field_name_lower_case(const orc::Type* orc_type, int Status OrcReader::get_next_block(Block* block, size_t* read_rows, bool* eof) { RETURN_IF_ERROR(get_next_block_impl(block, read_rows, eof)); + if (*eof) { + COUNTER_UPDATE(_orc_profile.selected_row_group_count, + _reader_metrics.SelectedRowGroupCount); + COUNTER_UPDATE(_orc_profile.evaluated_row_group_count, + _reader_metrics.EvaluatedRowGroupCount); + } if (_orc_filter) { RETURN_IF_ERROR(_orc_filter->get_status()); } diff --git a/be/src/vec/exec/format/orc/vorc_reader.h b/be/src/vec/exec/format/orc/vorc_reader.h index 0807f4949e58508..8c73957e79e4e0e 100644 --- a/be/src/vec/exec/format/orc/vorc_reader.h +++ b/be/src/vec/exec/format/orc/vorc_reader.h @@ -18,9 +18,9 @@ #pragma once #include -#include -#include +#include +#include #include #include #include @@ -51,6 +51,8 @@ #include "vec/exec/format/format_common.h" #include "vec/exec/format/generic_reader.h" #include "vec/exec/format/table/transactional_hive_reader.h" +#include "vec/exprs/vliteral.h" +#include "vec/exprs/vslot_ref.h" namespace doris { class RuntimeState; @@ -80,13 +82,6 @@ namespace doris::vectorized { class ORCFileInputStream; -struct OrcPredicate { - std::string col_name; - orc::PredicateDataType data_type; - std::vector literals; - SQLFilterOp op; -}; - struct LazyReadContext { VExprContextSPtrs conjuncts; bool can_lazy_read = false; @@ -228,6 +223,8 @@ class OrcReader : public GenericReader { RuntimeProfile::Counter* decode_value_time = nullptr; RuntimeProfile::Counter* decode_null_map_time = nullptr; RuntimeProfile::Counter* filter_block_time = nullptr; + RuntimeProfile::Counter* selected_row_group_count = nullptr; + RuntimeProfile::Counter* evaluated_row_group_count = nullptr; }; class ORCFilterImpl : public orc::ORCFilter { @@ -291,8 +288,23 @@ class OrcReader : public GenericReader { bool* is_hive1_orc); static bool _check_acid_schema(const orc::Type& type); static const orc::Type& _remove_acid(const orc::Type& type); - bool _init_search_argument( - std::unordered_map* colname_to_value_range); + std::tuple _make_orc_literal( + const VSlotRef* slot_ref, const VLiteral* literal); + bool _check_slot_can_push_down(const VExprSPtr& expr); + bool _check_rest_children_can_push_down(const VExprSPtr& expr); + bool _check_expr_can_push_down(const VExprSPtr& expr); + bool _build_less_than(const VExprSPtr& expr, + std::unique_ptr& builder); + bool _build_less_than_equals(const VExprSPtr& expr, + std::unique_ptr& builder); + bool _build_equals(const VExprSPtr& expr, std::unique_ptr& builder); + bool _build_filter_in(const VExprSPtr& expr, + std::unique_ptr& builder); + bool _build_is_null(const VExprSPtr& expr, + std::unique_ptr& builder); + bool _build_search_argument(const VExprSPtr& expr, + std::unique_ptr& builder); + bool _init_search_argument(const VExprContextSPtrs& conjuncts); void _init_bloom_filter( std::unordered_map* colname_to_value_range); void _init_system_properties(); @@ -578,11 +590,14 @@ class OrcReader : public GenericReader { bool _is_hive1_orc_or_use_idx = false; std::unordered_map _col_name_to_file_col_name; + // TODO: check if we can remove _col_name_to_file_col_name_low_case + std::unordered_map _col_name_to_file_col_name_low_case; std::unordered_map _type_map; std::vector _col_orc_type; std::unique_ptr _file_input_stream; Statistics _statistics; OrcProfile _orc_profile; + orc::ReaderMetrics _reader_metrics; std::unique_ptr _batch; std::unique_ptr _reader; diff --git a/be/test/exec/test_data/orc_scanner/orders.orc b/be/test/exec/test_data/orc_scanner/orders.orc new file mode 100644 index 0000000000000000000000000000000000000000..6fad5043288d430f6f158fd3efedf5050ad185c3 GIT binary patch literal 1293 zcmbtT&ubGw6n?WmXD7SqbWL@#ly;2~V_R+dBW(;Q!KjoX&C-hz#TeSC!8BWv&7mG7 zSn%Mosri|?xy;4cc{?6SJNruyCXs5W-w26vlzwJz`auSPAn%{Wn+AosGd(8tK;3{+? z01Y7SB7VstB6;W-0B?VO;<&g0d0gCs#cA9X-XH|MQGnf}rh5+_HpIJOJzPh|Kb``& zEj!;P(wR8u7DM-X zRw+f<#Bcl&>-nQ3tZg28Lfzh4{%ac~o+lyJ^Fe8RLWZu~y-Q`@RU8XnUnH?f+&3ge z5}SmYwm~FWQ8k?!4)e=?a}|AB-Ea)QywYw3y{WYew}ML0ZaJo34*Z}}UvJh{s!jxx zWHz>1398P3HFh6SzgY`5932UIo_C^tx%H^FJ`^YAq`$USZ3GF4%9fJX6B>=C%(R5- z0UWxe5O+sUlUyiD?vNre#pOc$7jX&ZivR!s literal 0 HcmV?d00001 diff --git a/be/test/testutil/desc_tbl_builder.cpp b/be/test/testutil/desc_tbl_builder.cpp index 4cba9a44a4b0d1b..6404d1c5449165d 100644 --- a/be/test/testutil/desc_tbl_builder.cpp +++ b/be/test/testutil/desc_tbl_builder.cpp @@ -17,20 +17,9 @@ #include "testutil/desc_tbl_builder.h" -#include -#include -#include +#include -#include - -#include "common/object_pool.h" #include "common/status.h" -#include "gtest/gtest_pred_impl.h" -#include "runtime/define_primitive_type.h" -#include "runtime/descriptors.h" -#include "util/bit_util.h" - -using std::vector; namespace doris { @@ -44,7 +33,7 @@ TupleDescBuilder& DescriptorTblBuilder::declare_tuple() { // item_id of -1 indicates no itemTupleId static TSlotDescriptor make_slot_descriptor(int id, int parent_id, const TypeDescriptor& type, - int slot_idx, int item_id) { + const std::string& name, int slot_idx, int item_id) { int null_byte = slot_idx / 8; int null_bit = slot_idx % 8; TSlotDescriptor slot_desc; @@ -58,6 +47,7 @@ static TSlotDescriptor make_slot_descriptor(int id, int parent_id, const TypeDes slot_desc.__set_nullIndicatorBit(null_bit); slot_desc.__set_slotIdx(slot_idx); slot_desc.__set_isMaterialized(true); + slot_desc.__set_colName(name); // if (item_id != -1) { // slot_desc.__set_itemTupleId(item_id); // } @@ -78,8 +68,9 @@ DescriptorTbl* DescriptorTblBuilder::build() { int tuple_id = 0; int slot_id = 0; - for (int i = 0; i < _tuples_descs.size(); ++i) { - build_tuple(_tuples_descs[i]->slot_types(), &thrift_desc_tbl, &tuple_id, &slot_id); + for (auto& _tuples_desc : _tuples_descs) { + build_tuple(_tuples_desc->slot_types(), _tuples_desc->slot_names(), &thrift_desc_tbl, + &tuple_id, &slot_id); } Status status = DescriptorTbl::create(_obj_pool, thrift_desc_tbl, &desc_tbl); @@ -87,7 +78,8 @@ DescriptorTbl* DescriptorTblBuilder::build() { return desc_tbl; } -TTupleDescriptor DescriptorTblBuilder::build_tuple(const vector& slot_types, +TTupleDescriptor DescriptorTblBuilder::build_tuple(const std::vector& slot_types, + const std::vector& slot_names, TDescriptorTable* thrift_desc_tbl, int* next_tuple_id, int* slot_id) { // We never materialize struct slots (there's no in-memory representation of structs, @@ -95,7 +87,8 @@ TTupleDescriptor DescriptorTblBuilder::build_tuple(const vector& // still have a struct item type. In this case, the array item tuple contains the // "inlined" struct fields. if (slot_types.size() == 1 && slot_types[0].type == TYPE_STRUCT) { - return build_tuple(slot_types[0].children, thrift_desc_tbl, next_tuple_id, slot_id); + return build_tuple(slot_types[0].children, slot_types[0].field_names, thrift_desc_tbl, + next_tuple_id, slot_id); } int tuple_id = *next_tuple_id; @@ -111,7 +104,7 @@ TTupleDescriptor DescriptorTblBuilder::build_tuple(const vector& // } thrift_desc_tbl->slotDescriptors.push_back( - make_slot_descriptor(*slot_id, tuple_id, slot_types[i], i, item_id)); + make_slot_descriptor(*slot_id, tuple_id, slot_types[i], slot_names[i], i, item_id)); thrift_desc_tbl->__isset.slotDescriptors = true; ++(*slot_id); } diff --git a/be/test/testutil/desc_tbl_builder.h b/be/test/testutil/desc_tbl_builder.h index c29ef9acd43eb51..968b29bd00143cc 100644 --- a/be/test/testutil/desc_tbl_builder.h +++ b/be/test/testutil/desc_tbl_builder.h @@ -20,15 +20,16 @@ #include +#include #include +#include "common/object_pool.h" +#include "runtime/descriptors.h" #include "runtime/types.h" namespace doris { -class ObjectPool; class TupleDescBuilder; -class DescriptorTbl; // Aids in the construction of a DescriptorTbl by declaring tuples and slots // associated with those tuples. @@ -40,6 +41,7 @@ class DescriptorTbl; // DescriptorTblBuilder builder; // builder.declare_tuple() << TYPE_TINYINT << TYPE_TIMESTAMP; // gets TupleId 0 // builder.declare_tuple() << TYPE_FLOAT; // gets TupleId 1 +// builder.declare_tuple() << std::make_tuple(TYPE_INT, "col1") << std::make_tuple(TYPE_STRING, "col2"); // gets Tuple with type and name // DescriptorTbl desc_tbl = builder.build(); class DescriptorTblBuilder { public: @@ -57,20 +59,31 @@ class DescriptorTblBuilder { std::vector _tuples_descs; TTupleDescriptor build_tuple(const std::vector& slot_types, + const std::vector& slot_names, TDescriptorTable* thrift_desc_tbl, int* tuple_id, int* slot_id); }; class TupleDescBuilder { public: + using SlotType = std::tuple; + TupleDescBuilder& operator<<(const SlotType& slot) { + _slot_types.push_back(std::get<0>(slot)); + _slot_names.push_back(std::get<1>(slot)); + return *this; + } + TupleDescBuilder& operator<<(const TypeDescriptor& slot_type) { _slot_types.push_back(slot_type); + _slot_names.emplace_back(""); return *this; } std::vector slot_types() const { return _slot_types; } + std::vector slot_names() const { return _slot_names; } private: std::vector _slot_types; + std::vector _slot_names; }; } // end namespace doris diff --git a/be/test/vec/exec/orc_reader_test.cpp b/be/test/vec/exec/orc_reader_test.cpp new file mode 100644 index 000000000000000..ec5bd9b519de633 --- /dev/null +++ b/be/test/vec/exec/orc_reader_test.cpp @@ -0,0 +1,155 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include +#include + +#include +#include +#include +#include + +#include "orc/sargs/SearchArgument.hh" +#include "runtime/define_primitive_type.h" +#include "runtime/exec_env.h" +#include "runtime/runtime_state.h" +#include "testutil/desc_tbl_builder.h" +#include "vec/exec/format/orc/orc_memory_pool.h" +#include "vec/exec/format/orc/vorc_reader.h" +#include "vec/exprs/vexpr_context.h" +#include "vec/exprs/vexpr_fwd.h" +#include "vec/utils/util.hpp" +namespace doris::vectorized { +class OrcReaderTest : public testing::Test { +public: + OrcReaderTest() = default; + ~OrcReaderTest() override = default; + +private: + static constexpr const char* CANNOT_PUSH_DOWN_ERROR = "can't push down"; + std::string build_search_argument(const std::string& expr) { + // build orc_reader for table orders + std::vector column_names = { + "o_orderkey", "o_custkey", "o_orderstatus", "o_totalprice", "o_orderdate", + "o_orderpriority", "o_clerk", "o_shippriority", "o_comment"}; + ObjectPool object_pool; + DescriptorTblBuilder builder(&object_pool); + builder.declare_tuple() << std::make_tuple(TYPE_INT, "o_orderkey") + << std::make_tuple(TYPE_INT, "o_custkey") + << std::make_tuple(TYPE_STRING, "o_orderstatus") + << std::make_tuple(TYPE_DOUBLE, "o_totalprice") + << std::make_tuple(TYPE_DATE, "o_orderdate") + << std::make_tuple(TYPE_STRING, "o_orderpriority") + << std::make_tuple(TYPE_STRING, "o_clerk") + << std::make_tuple(TYPE_INT, "o_shippriority") + << std::make_tuple(TYPE_STRING, "o_comment"); + DescriptorTbl* desc_tbl = builder.build(); + auto* tuple_desc = const_cast(desc_tbl->get_tuple_descriptor(0)); + RowDescriptor row_desc(tuple_desc, false); + TFileScanRangeParams params; + TFileRangeDesc range; + range.path = "./be/test/exec/test_data/orc_scanner/orders.orc"; + range.start_offset = 0; + range.size = 1293; + auto reader = OrcReader::create_unique(params, range, "", nullptr, true); + auto status = reader->init_reader(&column_names, nullptr, {}, false, tuple_desc, &row_desc, + nullptr, nullptr); + EXPECT_TRUE(status.ok()); + + // deserialize expr + auto exprx = apache::thrift::from_json_string(expr); + VExprContextSPtr context; + status = VExpr::create_expr_tree(exprx, context); + EXPECT_TRUE(status.ok()); + + // prepare expr context + RuntimeState state; + state.set_desc_tbl(desc_tbl); + status = context->prepare(&state, row_desc); + EXPECT_TRUE(status.ok()); + + // build search argument + auto sarg_builder = orc::SearchArgumentFactory::newBuilder(); + auto res = reader->_build_search_argument(context->root(), sarg_builder); + if (!res) { + return CANNOT_PUSH_DOWN_ERROR; + } + return sarg_builder->build()->toString(); + } +}; + +TEST_F(OrcReaderTest, test_build_search_argument) { + ExecEnv::GetInstance()->set_orc_memory_pool(new ORCMemoryPool()); + std:: + vector + exprs = + { + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey < 100 or o_orderkey > 5999900 or o_orderkey in (1000000, 2000000, 3000000); + R"|({"1":{"lst":["rec",13,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":100}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":13},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"gt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"gt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":5999900}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":11},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":5},"4":{"i32":4},"11":{"rec":{"1":{"tf":0}}},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":3000000}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey is null or (o_orderkey between 100 and 1000 and o_orderkey not in (200, 300, 400)); + R"|({"1":{"lst":["rec",16,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"is_null_pred"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"is_null_pred(int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":0}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":100}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":12},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"le"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"le(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":11},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":6},"4":{"i32":4},"11":{"rec":{"1":{"tf":1}}},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":200}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":300}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":400}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey is null or (o_orderkey between 1000000 and 1200000 and o_orderkey != 1100000); + R"|({"1":{"lst":["rec",14,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"is_null_pred"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"is_null_pred(int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":0}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":12},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"le"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"le(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1200000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":10},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ne"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ne(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1100000}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // SELECT count(o_orderkey) FROM tpch1_orc.orders WHERE o_orderkey IN (1000000, 2000000, 3000000) OR (o_orderdate >= '1994-01-01' AND o_orderdate <= '1994-12-31'); + R"|({"1":{"lst":["rec",13,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":11},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":5},"4":{"i32":4},"11":{"rec":{"1":{"tf":0}}},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":3000000}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(datev2, datev2)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":26},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":4},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderdate"}},{"1":{"i32":7},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"8":{"rec":{"1":{"str":"1994-01-01"}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":12},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"le"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"le(datev2, datev2)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":26},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":4},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderdate"}},{"1":{"i32":7},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"8":{"rec":{"1":{"str":"1994-12-31"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey < 2 or (o_comment like '%delayed%' and o_orderpriority = '1-URGENT'); + R"|({"1":{"lst":["rec",11,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"like"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":9}}}}]},"3":{"i64":9}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"like(text, varchar(9))"},"9":{"rec":{"1":{"str":""}}},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":8},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_comment"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"%delayed%"}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":9},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"eq"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"eq(text, varchar(65533))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":23},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":5},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderpriority"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"1-URGENT"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey < 1 + 1; + R"|({"1":{"lst":["rec",3,{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // SELECT count(o_orderkey) FROM tpch1_orc.orders WHERE o_comment LIKE '%delayed%' OR o_orderpriority = '1-URGENT'; + R"|({"1":{"lst":["rec",7,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"like"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":9}}}}]},"3":{"i64":9}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"like(text, varchar(9))"},"9":{"rec":{"1":{"str":""}}},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":8},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_comment"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"%delayed%"}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":9},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"eq"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"eq(text, varchar(65533))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":23},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":5},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderpriority"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"1-URGENT"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey between 1 and 100 or random() > 0.5; + R"|({"1":{"lst":["rec",11,{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":3},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":6},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":2},"4":{"i32":2},"20":{"i32":-1},"29":{"tf":1}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":14},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"ge"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"ge(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":1}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":12},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"le"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"le(int, int)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":5},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":100}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":13},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"gt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"gt(double, double)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":8},"29":{"tf":0}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"random"}}},"2":{"i32":0},"3":{"lst":["rec",0]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"random()"},"9":{"rec":{"1":{"str":""}}},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":0}},{"1":{"i32":8},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"9":{"rec":{"1":{"dbl":0.5}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where lower(o_orderpriority) = '1-urgent'; + R"|({"1":{"lst":["rec",4,{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":9},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"eq"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"eq(varchar(65533), varchar(65533))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":15},"29":{"tf":1}},{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lower"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lower(text)"},"9":{"rec":{"1":{"str":""}}},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":23},"2":{"i32":2147483643}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":5},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderpriority"}},{"1":{"i32":17},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":15},"2":{"i32":65533}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"16":{"rec":{"1":{"str":"1-urgent"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderkey * 2 < 60; + R"|({"1":{"lst":["rec",5,{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":11},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"lt"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"lt(bigint, bigint)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":6},"29":{"tf":1}},{"1":{"i32":1},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}}},"3":{"i32":55},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"multiply"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":3}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"multiply(int, tinyint)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderkey"}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":3}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":2}}},"20":{"i32":-1},"29":{"tf":0}},{"1":{"i32":9},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"10":{"rec":{"1":{"i64":60}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + // select count(o_orderkey) from tpch1_orc.orders where o_orderdate is not null; + R"|({"1":{"lst":["rec",4,{"1":{"i32":2},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"3":{"i32":9},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"eq"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":2}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"eq(datetimev2(0), datetimev2(0))"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"28":{"i32":27},"29":{"tf":1}},{"1":{"i32":5},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}}},"3":{"i32":4},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"casttodatetimev2"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}}},"5":{"tf":0},"7":{"str":"casttodatetimev2(datev2)"},"11":{"i64":0},"13":{"tf":1},"14":{"tf":0},"15":{"tf":0},"16":{"i64":360}}},"29":{"tf":1}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":26}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":4},"2":{"i32":0},"3":{"i32":-1}}},"20":{"i32":-1},"29":{"tf":1},"36":{"str":"o_orderdate"}},{"1":{"i32":7},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":27},"3":{"i32":18},"4":{"i32":0}}}}]},"3":{"i64":-1}}},"4":{"i32":0},"8":{"rec":{"1":{"str":"2024-11-12 21:13:02"}}},"20":{"i32":-1},"29":{"tf":0}}]}})|", + }; + std::vector result_search_arguments = { + "leaf-0 = (o_orderkey < 100), leaf-1 = (o_orderkey <= 5999900), leaf-2 " + "= (o_orderkey " + "in " + "[1000000, 2000000, 3000000]), expr = (or leaf-0 (not leaf-1) leaf-2)", + "leaf-0 = (o_orderkey is null), leaf-1 = (o_orderkey < 100), leaf-2 = " + "(o_orderkey <= " + "1000), leaf-3 = (o_orderkey in [200, 300, 400]), expr = (and (or " + "leaf-0 (not leaf-1)) " + "(or leaf-0 leaf-2) (or leaf-0 (not leaf-3)))", + "leaf-0 = (o_orderkey is null), leaf-1 = (o_orderkey < 1000000), leaf-2 = (o_orderkey " + "<= 1200000), leaf-3 = (o_orderkey = 1100000), expr = (and (or leaf-0 (not leaf-1)) " + "(or leaf-0 leaf-2) (or leaf-0 (not leaf-3)))", + "leaf-0 = (o_orderkey in [1000000, 2000000, 3000000]), leaf-1 = (o_orderdate < " + "17121205), leaf-2 = (o_orderdate <= 17121205), expr = (and (or leaf-0 (not leaf-1)) " + "(or leaf-0 leaf-2))", + "leaf-0 = (o_orderkey < 2), leaf-1 = (o_orderpriority = 1-URGENT), expr = (or leaf-0 " + "leaf-1)", + "leaf-0 = (o_orderkey < 2), expr = leaf-0", + CANNOT_PUSH_DOWN_ERROR, + CANNOT_PUSH_DOWN_ERROR, + CANNOT_PUSH_DOWN_ERROR, + CANNOT_PUSH_DOWN_ERROR, + CANNOT_PUSH_DOWN_ERROR, + }; + for (int i = 0; i < exprs.size(); i++) { + auto search_argument = build_search_argument(exprs[i]); + ASSERT_EQ(search_argument, result_search_arguments[i]); + } +} + +} // namespace doris::vectorized diff --git a/regression-test/data/external_table_p0/hive/test_hive_orc.out b/regression-test/data/external_table_p0/hive/test_hive_orc.out index 066c5d4b4d3b5f9..b34f276020c7cc1 100644 --- a/regression-test/data/external_table_p0/hive/test_hive_orc.out +++ b/regression-test/data/external_table_p0/hive/test_hive_orc.out @@ -110,6 +110,30 @@ tablets tinyint_col 179 182 182 187 183 181 177 183 177 187 183 202 202 186 528 -- !string_col_dict_plain_mixed3 -- 10240 +-- !predicate_pushdown1 -- +55 + +-- !predicate_pushdown2 -- +228 + +-- !predicate_pushdown3 -- +53 + +-- !predicate_pushdown4 -- +50000 + +-- !predicate_pushdown5 -- +90425 + +-- !predicate_pushdown6 -- +279428 + +-- !predicate_pushdown7 -- +300343 + +-- !predicate_pushdown8 -- +1533 + -- !select_top50 -- 4 55 999742610 400899305488827731 false 6.5976813E8 7.8723304616937395E17 \N base tennis pit vertical friday 2022-08-19T07:29:58 \N tablets smallint_col 2019-02-07 [7.53124931825377e+17] ["NbSSBtwzpxNSkkwga"] tablets smallint_col 2 49 999613702 105493714032727452 \N 6.3322381E8 9.8642324410240179E17 Unveil bright recruit participate. Suspect impression camera mathematical revelation. Fault live2 elbow debt west hydrogen current. how literary 2022-09-03T17:20:21 481707.1065 tablets boolean_col 2020-01-12 [] ["HoMrAnn", "wteEFvIwoZsVpVQdscMb", null, "zcGFmv", "kGEBBckbMtX", "hrEtCGFdPWZK"] tablets boolean_col @@ -273,6 +297,30 @@ tablets tinyint_col 179 182 182 187 183 181 177 183 177 187 183 202 202 186 528 -- !string_col_dict_plain_mixed3 -- 10240 +-- !predicate_pushdown1 -- +55 + +-- !predicate_pushdown2 -- +228 + +-- !predicate_pushdown3 -- +53 + +-- !predicate_pushdown4 -- +50000 + +-- !predicate_pushdown5 -- +90425 + +-- !predicate_pushdown6 -- +279428 + +-- !predicate_pushdown7 -- +300343 + +-- !predicate_pushdown8 -- +1533 + -- !select_top50 -- 4 55 999742610 400899305488827731 false 6.5976813E8 7.8723304616937395E17 \N base tennis pit vertical friday 2022-08-19T07:29:58 \N tablets smallint_col 2019-02-07 [7.53124931825377e+17] ["NbSSBtwzpxNSkkwga"] tablets smallint_col 2 49 999613702 105493714032727452 \N 6.3322381E8 9.8642324410240179E17 Unveil bright recruit participate. Suspect impression camera mathematical revelation. Fault live2 elbow debt west hydrogen current. how literary 2022-09-03T17:20:21 481707.1065 tablets boolean_col 2020-01-12 [] ["HoMrAnn", "wteEFvIwoZsVpVQdscMb", null, "zcGFmv", "kGEBBckbMtX", "hrEtCGFdPWZK"] tablets boolean_col diff --git a/regression-test/suites/external_table_p0/hive/test_hive_orc.groovy b/regression-test/suites/external_table_p0/hive/test_hive_orc.groovy index 0f837c0abd3088a..6457d2b3edd5d82 100644 --- a/regression-test/suites/external_table_p0/hive/test_hive_orc.groovy +++ b/regression-test/suites/external_table_p0/hive/test_hive_orc.groovy @@ -81,6 +81,17 @@ suite("test_hive_orc", "all_types,p0,external,hive,external_docker,external_dock qt_string_col_dict_plain_mixed3 """select count(col2) from string_col_dict_plain_mixed_orc where col1 like '%Test%';""" } + def predicate_pushdown = { + qt_predicate_pushdown1 """ select count(o_orderkey) from tpch1_orc.orders where o_orderkey is not null and (o_orderkey < 100 or o_orderkey > 5999900 or o_orderkey in (1000000, 2000000, 3000000)); """ + qt_predicate_pushdown2 """ select count(o_orderkey) from tpch1_orc.orders where o_orderkey is null or (o_orderkey between 100 and 1000 and o_orderkey not in (200, 300, 400)); """ + qt_predicate_pushdown3 """ select count(o_orderkey) from tpch1_orc.orders where o_orderkey is not null and (o_orderkey < 100 or o_orderkey > 5999900 or o_orderkey = 3000000); """ + qt_predicate_pushdown4 """ select count(o_orderkey) from tpch1_orc.orders where o_orderkey is null or (o_orderkey between 1000000 and 1200000 and o_orderkey != 1100000); """ + qt_predicate_pushdown5 """ SELECT count(o_orderkey) FROM tpch1_orc.orders WHERE (o_orderdate >= '1994-01-01' AND o_orderdate <= '1994-12-31') AND (o_orderpriority = '5-LOW' OR o_orderpriority = '3-MEDIUM') AND o_totalprice > 2000;""" + qt_predicate_pushdown6 """ SELECT count(o_orderkey) FROM tpch1_orc.orders WHERE o_orderstatus <> 'F' AND o_custkey < 54321; """ + qt_predicate_pushdown7 """ SELECT count(o_orderkey) FROM tpch1_orc.orders WHERE o_comment LIKE '%delayed%' OR o_orderpriority = '1-URGENT'; """ + qt_predicate_pushdown8 """ SELECT count(o_orderkey) FROM tpch1_orc.orders WHERE o_orderkey IN (1000000, 2000000, 3000000) OR o_clerk = 'Clerk#000000470'; """ + } + String enabled = context.config.otherConfigs.get("enableHiveTest") if (enabled == null || !enabled.equalsIgnoreCase("true")) { logger.info("diable Hive test.") @@ -108,6 +119,7 @@ suite("test_hive_orc", "all_types,p0,external,hive,external_docker,external_dock only_partition_col() decimals() string_col_dict_plain_mixed() + predicate_pushdown() sql """drop catalog if exists ${catalog_name}""" From f6ea841d8c23782c92d5f4469c8706ec1e9ef126 Mon Sep 17 00:00:00 2001 From: zfr95 Date: Fri, 22 Nov 2024 10:21:25 +0800 Subject: [PATCH 030/110] [test](auth)Add grant show_view_priv to new user (#44260) --- .../test_account_management_user_auth.groovy | 3 +- .../test_grant_show_view_priv_auth.groovy | 94 +++++++++++++++++++ 2 files changed, 96 insertions(+), 1 deletion(-) create mode 100644 regression-test/suites/auth_call/test_grant_show_view_priv_auth.groovy diff --git a/regression-test/suites/auth_call/test_account_management_user_auth.groovy b/regression-test/suites/auth_call/test_account_management_user_auth.groovy index e6b0c203dd367e9..936071b1e9d9ef6 100644 --- a/regression-test/suites/auth_call/test_account_management_user_auth.groovy +++ b/regression-test/suites/auth_call/test_account_management_user_auth.groovy @@ -23,7 +23,7 @@ suite("test_account_management_user_auth","p0,auth_call") { String user = 'test_account_management_user_auth_user' String pwd = 'C123_567p' String dbName = 'test_account_management_user_auth_db' - String user_derive = 'test_account_management_user_derive_role' + String user_derive = 'test_account_management_user_derive_user' //cloud-mode if (isCloudMode()) { @@ -68,6 +68,7 @@ suite("test_account_management_user_auth","p0,auth_call") { } } sql """grant grant_priv on *.*.* to '${user}'""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { sql """CREATE USER ${user_derive} IDENTIFIED BY '${pwd}';""" sql """ALTER USER ${user_derive} IDENTIFIED BY "${pwd}";""" diff --git a/regression-test/suites/auth_call/test_grant_show_view_priv_auth.groovy b/regression-test/suites/auth_call/test_grant_show_view_priv_auth.groovy new file mode 100644 index 000000000000000..59a7dc1efa8b5ef --- /dev/null +++ b/regression-test/suites/auth_call/test_grant_show_view_priv_auth.groovy @@ -0,0 +1,94 @@ +// 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. + +import org.junit.Assert; +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite("test_grant_show_view_priv_auth","p0,auth_call") { + + String user = 'test_grant_show_view_priv_user' + String pwd = 'C123_567p' + String dbName = 'test_grant_show_view_priv_auth_db' + String user_derive = 'test_grant_show_view_priv_derive_user' + String tableName = 'test_grant_show_view_priv_auth_tb' + String viewName = 'test_grant_show_view_priv_auth_view' + + //cloud-mode + if (isCloudMode()) { + def clusters = sql " SHOW CLUSTERS; " + assertTrue(!clusters.isEmpty()) + def validCluster = clusters[0][0] + sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}"""; + } + + try_sql("DROP USER ${user}") + try_sql("DROP USER ${user_derive}") + try_sql """drop database if exists ${dbName}""" + + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + sql """create database ${dbName}""" + sql """create table ${dbName}.${tableName} ( + id BIGINT, + username VARCHAR(20) + ) + DISTRIBUTED BY HASH(id) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + );""" + sql """ + INSERT INTO ${dbName}.${tableName} (id, username) + VALUES (1, "111"), + (2, "222"), + (3, "333") + """ + sql """CREATE VIEW ${dbName}.${viewName} (k1, v1) + AS + SELECT id as k1, SUM(id) FROM ${dbName}.${tableName} + WHERE id = 1 GROUP BY k1; + """ + + sql """grant grant_priv on *.*.* to '${user}'""" + sql """grant SHOW_VIEW_PRIV on ${dbName}.${viewName} to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + sql """CREATE USER ${user_derive} IDENTIFIED BY '${pwd}';""" + sql """grant select_priv on regression_test to ${user_derive}""" + + connect(user=user_derive, password="${pwd}", url=context.config.jdbcUrl) { + test { + sql """show create table ${dbName}.${viewName};""" + exception "denied" + } + test { + sql """select * from ${dbName}.${viewName};""" + exception "denied" + } + } + sql """grant SHOW_VIEW_PRIV on ${dbName}.${viewName} to ${user_derive}""" + connect(user=user_derive, password="${pwd}", url=context.config.jdbcUrl) { + sql """show create table ${dbName}.${viewName};""" + test { + sql """select * from ${dbName}.${viewName};""" + exception "denied" + } + } + } + + sql """drop database if exists ${dbName}""" + try_sql("DROP USER ${user}") + try_sql("DROP role ${user_derive}") +} From 589878ac5f0ffbc2f2aa437dcd09951fdfc68151 Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Fri, 22 Nov 2024 11:00:48 +0800 Subject: [PATCH 031/110] [Fix](Job)The INSERT execution failed, but the task record status is marked as successful. (#44292) ### What problem does this PR solve? After a job execution is completed, we need to verify whether the QueryState is normal. Currently, we rely on exception handling to determine success. However, in certain cases, such as execution timeouts, exceptions may not be captured. As a result, the status is incorrectly marked as successful. ### Release note None ### Check List (For Author) - Test - [x] Manual test (add detailed scripts or steps below) ``` select * from tasks('type'='insert') ; +----------------+----------------+---------+-------------------------------+---------+----------------------------------------------+---------------------+---------------------+------------+-------------+---------------+------+ | TaskId | JobId | JobName | Label | Status | ErrorMsg | CreateTime | StartTime | FinishTime | TrackingUrl | LoadStatistic | User | +----------------+----------------+---------+-------------------------------+---------+----------------------------------------------+---------------------+---------------------+------------+-------------+---------------+------+ | 53767413936871 | 53748267972932 | test | 53748267972932_53767413936871 | RUNNING | | 2024-11-19 21:39:46 | 2024-11-19 21:39:46 | | | | root | | 53758617801828 | 53748267972932 | test | 53748267972932_53758617801828 | FAILED | errCode = 2, detailMessage = Execute timeout | 2024-11-19 21:39:37 | 2024-11-19 21:39:37 | | | | root | ``` --- .../job/extensions/insert/InsertTask.java | 30 ++++++++++++------- 1 file changed, 19 insertions(+), 11 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java index d1a425aeaf78382..c997ebcd30e7757 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java @@ -23,6 +23,7 @@ import org.apache.doris.catalog.ScalarType; import org.apache.doris.common.Status; import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.common.util.Util; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.task.AbstractTask; import org.apache.doris.load.FailMsg; @@ -31,6 +32,7 @@ import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.QueryState; import org.apache.doris.qe.StmtExecutor; import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; @@ -195,10 +197,13 @@ public void run() throws JobException { return; } command.runWithUpdateInfo(ctx, stmtExecutor, loadStatistic); + if (ctx.getState().getStateType() != QueryState.MysqlStateType.OK) { + throw new JobException(ctx.getState().getErrorMessage()); + } } catch (Exception e) { log.warn("execute insert task error, job id is {}, task id is {},sql is {}", getJobId(), getTaskId(), sql, e); - throw new JobException(e); + throw new JobException(Util.getRootCauseMessage(e)); } } @@ -237,15 +242,7 @@ public TRow getTvfInfo(String jobName) { trow.addToColumnValue(new TCell().setStringVal(jobName)); trow.addToColumnValue(new TCell().setStringVal(getJobId() + LABEL_SPLITTER + getTaskId())); trow.addToColumnValue(new TCell().setStringVal(jobInfo.getState().name())); - // err msg - String errorMsg = ""; - if (failMsg != null) { - errorMsg = failMsg.getMsg(); - } - if (StringUtils.isNotBlank(getErrMsg())) { - errorMsg = getErrMsg(); - } - trow.addToColumnValue(new TCell().setStringVal(errorMsg)); + trow.addToColumnValue(new TCell().setStringVal(getErrorMsg())); // create time trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(getCreateTimeMs()))); trow.addToColumnValue(new TCell().setStringVal(null == getStartTimeMs() ? "" @@ -275,7 +272,7 @@ private TRow getPendingTaskTVFInfo(String jobName) { trow.addToColumnValue(new TCell().setStringVal(jobName)); trow.addToColumnValue(new TCell().setStringVal(getJobId() + LABEL_SPLITTER + getTaskId())); trow.addToColumnValue(new TCell().setStringVal(getStatus().name())); - trow.addToColumnValue(new TCell().setStringVal("")); + trow.addToColumnValue(new TCell().setStringVal(getErrorMsg())); trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(getCreateTimeMs()))); trow.addToColumnValue(new TCell().setStringVal(null == getStartTimeMs() ? "" : TimeUtils.longToTimeString(getStartTimeMs()))); @@ -287,4 +284,15 @@ private TRow getPendingTaskTVFInfo(String jobName) { return trow; } + private String getErrorMsg() { + // err msg + String errorMsg = ""; + if (failMsg != null) { + errorMsg = failMsg.getMsg(); + } + if (StringUtils.isNotBlank(getErrMsg())) { + errorMsg = getErrMsg(); + } + return errorMsg; + } } From bc679038ca8730afa63a8d3b83ce7c34c8985e63 Mon Sep 17 00:00:00 2001 From: Pxl Date: Fri, 22 Nov 2024 11:36:12 +0800 Subject: [PATCH 032/110] [Bug](runtime-filter) avoid ignore rf multiple times (#44408) ### What problem does this PR solve? avoid ignore rf multiple times When an instance is wake_up_by_downstream, it will destroy the logic of `first closing the instance that should build hash table and then closing other instances`, instance that should build hash table during the process of inserting data into rf, this instance may find that rf is ignored, causing dcheck to fail. ```cpp F20241121 12:01:02.245405 9832 runtime_filter.cpp:395] Check failed: !is_ignored() 3# raise at ../sysdeps/posix/raise.c:27 4# abort at ./stdlib/abort.c:81 5# 0x00005603CFDF778D in /mnt/ssd01/doris-branch40preview/NEREIDS_ASAN/be/lib/doris_be 6# 0x00005603CFDE9DCA in /mnt/ssd01/doris-branch40preview/NEREIDS_ASAN/be/lib/doris_be 7# google::LogMessage::SendToLog() in /mnt/ssd01/doris-branch40preview/NEREIDS_ASAN/be/lib/doris_be 8# google::LogMessage::Flush() in /mnt/ssd01/doris-branch40preview/NEREIDS_ASAN/be/lib/doris_be 9# google::LogMessageFatal::~LogMessageFatal() in /mnt/ssd01/doris-branch40preview/NEREIDS_ASAN/be/lib/doris_be 10# doris::RuntimePredicateWrapper::insert_fixed_len(COW::immutable_ptr const&, unsigned long) at /home/zcp/repo_center/doris_branch-3.0/doris/be/src/exprs/runtime_filter.cpp:395 11# doris::RuntimePredicateWrapper::insert_batch(COW::immutable_ptr const&, unsigned long) at /home/zcp/repo_center/doris_branch-3.0/doris/be/src/exprs/runtime_filter.cpp:431 12# doris::IRuntimeFilter::insert_batch(COW::immutable_ptr, unsigned long) at /home/zcp/repo_center/doris_branch-3.0/doris/be/src/exprs/runtime_filter.cpp:992 13# doris::VRuntimeFilterSlots::insert(doris::vectorized::Block const*) at /home/zcp/repo_center/doris_branch-3.0/doris/be/src/exprs/runtime_filter_slots.h:146 14# doris::pipeline::HashJoinBuildSinkLocalState::close(doris::RuntimeState*, doris::Status) in /mnt/ssd01/doris-branch40preview/NEREIDS_ASAN/be/lib/doris_be 15# doris::pipeline::DataSinkOperatorXBase::close(doris::RuntimeState*, doris::Status) in /mnt/ssd01/doris-branch40preview/NEREIDS_ASAN/be/lib/doris_be 16# doris::pipeline::PipelineTask::close(doris::Status) at /home/zcp/repo_center/doris_branch-3.0/doris/be/src/pipeline/pipeline_task.cpp:487 ``` --- be/src/pipeline/exec/hashjoin_build_sink.cpp | 55 ++++++++++---------- 1 file changed, 28 insertions(+), 27 deletions(-) diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index bf177742ab35638..7d3f4da935099ee 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -112,23 +112,24 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu if (_closed) { return Status::OK(); } - auto p = _parent->cast(); Defer defer {[&]() { - if (_should_build_hash_table) { - // The build side hash key column maybe no need output, but we need to keep the column in block - // because it is used to compare with probe side hash key column - if (p._should_keep_hash_key_column && _build_col_ids.size() == 1) { - p._should_keep_column_flags[_build_col_ids[0]] = true; - } + if (!_should_build_hash_table) { + return; + } + // The build side hash key column maybe no need output, but we need to keep the column in block + // because it is used to compare with probe side hash key column + auto p = _parent->cast(); + if (p._should_keep_hash_key_column && _build_col_ids.size() == 1) { + p._should_keep_column_flags[_build_col_ids[0]] = true; + } - if (_shared_state->build_block) { - // release the memory of unused column in probe stage - _shared_state->build_block->clear_column_mem_not_keep( - p._should_keep_column_flags, bool(p._shared_hashtable_controller)); - } + if (_shared_state->build_block) { + // release the memory of unused column in probe stage + _shared_state->build_block->clear_column_mem_not_keep( + p._should_keep_column_flags, bool(p._shared_hashtable_controller)); } - if (_should_build_hash_table && p._shared_hashtable_controller) { + if (p._shared_hashtable_controller) { p._shared_hashtable_controller->signal_finish(p.node_id()); } }}; @@ -137,22 +138,22 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu return Base::close(state, exec_status); } - if (state->get_task()->wake_up_by_downstream()) { - RETURN_IF_ERROR(_runtime_filter_slots->send_filter_size(state, 0, _finish_dependency)); - RETURN_IF_ERROR(_runtime_filter_slots->ignore_all_filters()); - } else { - auto* block = _shared_state->build_block.get(); - uint64_t hash_table_size = block ? block->rows() : 0; - { - SCOPED_TIMER(_runtime_filter_init_timer); - if (_should_build_hash_table) { + if (_should_build_hash_table) { + if (state->get_task()->wake_up_by_downstream()) { + RETURN_IF_ERROR(_runtime_filter_slots->send_filter_size(state, 0, _finish_dependency)); + RETURN_IF_ERROR(_runtime_filter_slots->ignore_all_filters()); + } else { + auto* block = _shared_state->build_block.get(); + uint64_t hash_table_size = block ? block->rows() : 0; + { + SCOPED_TIMER(_runtime_filter_init_timer); RETURN_IF_ERROR(_runtime_filter_slots->init_filters(state, hash_table_size)); + RETURN_IF_ERROR(_runtime_filter_slots->ignore_filters(state)); + } + if (hash_table_size > 1) { + SCOPED_TIMER(_runtime_filter_compute_timer); + _runtime_filter_slots->insert(block); } - RETURN_IF_ERROR(_runtime_filter_slots->ignore_filters(state)); - } - if (_should_build_hash_table && hash_table_size > 1) { - SCOPED_TIMER(_runtime_filter_compute_timer); - _runtime_filter_slots->insert(block); } } SCOPED_TIMER(_publish_runtime_filter_timer); From 288d109a9c5f3961c775e7ffdfd2fca3923ae8f0 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Fri, 22 Nov 2024 09:08:06 +0530 Subject: [PATCH 033/110] [Enhancement] (nereids)implement showTriggersCommand in nereids (#44278) Issue Number: close #42731 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 8 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/ShowTriggersCommand.java | 66 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../show/test_show_commands_nereids.groovy | 2 + 6 files changed, 83 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTriggersCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 177eff9b6db5432..ea2958c6bc9f970 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -224,6 +224,7 @@ supportedShowStatement | SHOW CREATE MATERIALIZED VIEW mvName=identifier ON tableName=multipartIdentifier #showCreateMaterializedView | SHOW BACKENDS #showBackends + | SHOW FULL? TRIGGERS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTriggers | SHOW FRONTENDS name=identifier? #showFrontends | SHOW TABLE tableId=INTEGER_VALUE #showTableId | SHOW WHITELIST #showWhitelist @@ -270,7 +271,6 @@ unsupportedShowStatement | SHOW FULL? VIEWS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showViews | SHOW FULL? PROCESSLIST #showProcessList | SHOW (GLOBAL | SESSION | LOCAL)? STATUS wildWhere? #showStatus - | SHOW FULL? TRIGGERS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTriggers | SHOW EVENTS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showEvents | SHOW CREATE VIEW name=multipartIdentifier #showCreateView | SHOW CREATE MATERIALIZED VIEW name=multipartIdentifier #showMaterializedView diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index d24e4c1fa7bb99b..d3b63541d10df26 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -225,6 +225,7 @@ import org.apache.doris.nereids.DorisParser.ShowTableIdContext; import org.apache.doris.nereids.DorisParser.ShowTabletsBelongContext; import org.apache.doris.nereids.DorisParser.ShowTrashContext; +import org.apache.doris.nereids.DorisParser.ShowTriggersContext; import org.apache.doris.nereids.DorisParser.ShowVariablesContext; import org.apache.doris.nereids.DorisParser.ShowViewContext; import org.apache.doris.nereids.DorisParser.ShowWhitelistContext; @@ -488,6 +489,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTabletsBelongCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTrashCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowTriggersCommand; import org.apache.doris.nereids.trees.plans.commands.ShowVariablesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowWhiteListCommand; @@ -4216,6 +4218,7 @@ public LogicalPlan visitShowBackends(ShowBackendsContext ctx) { return new ShowBackendsCommand(); } + @Override public LogicalPlan visitShowPlugins(ShowPluginsContext ctx) { return new ShowPluginsCommand(); } @@ -4229,6 +4232,11 @@ public LogicalPlan visitShowSqlBlockRule(ShowSqlBlockRuleContext ctx) { return new ShowSqlBlockRuleCommand(ruleName); } + @Override + public LogicalPlan visitShowTriggers(ShowTriggersContext ctx) { + return new ShowTriggersCommand(); + } + @Override public LogicalPlan visitShowRepositories(ShowRepositoriesContext ctx) { return new ShowRepositoriesCommand(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index c3d7fc70801447e..3a4484f81d01c4f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -202,6 +202,7 @@ public enum PlanType { SHOW_STORAGE_ENGINES_COMMAND, SHOW_TABLE_ID_COMMAND, SHOW_TRASH_COMMAND, + SHOW_TRIGGERS_COMMAND, SHOW_VARIABLES_COMMAND, SHOW_AUTHORS_COMMAND, SHOW_VIEW_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTriggersCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTriggersCommand.java new file mode 100644 index 000000000000000..dd9d2c18c577ee6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTriggersCommand.java @@ -0,0 +1,66 @@ +// 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.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * show triggers command + */ +public class ShowTriggersCommand extends ShowCommand { + private static final ShowResultSetMetaData META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("Trigger", ScalarType.createVarchar(64))) + .addColumn(new Column("Event", ScalarType.createVarchar(10))) + .addColumn(new Column("Table", ScalarType.createVarchar(80))) + .addColumn(new Column("Statement", ScalarType.createVarchar(64))) + .addColumn(new Column("Timing", ScalarType.createVarchar(80))) + .addColumn(new Column("Created", ScalarType.createVarchar(80))) + .addColumn(new Column("sql_mode", ScalarType.createVarchar(80))) + .addColumn(new Column("Definer", ScalarType.createVarchar(80))) + .addColumn(new Column("character_set_client", ScalarType.createVarchar(80))) + .addColumn(new Column("collation_connection", ScalarType.createVarchar(80))) + .addColumn(new Column("Database Collation", ScalarType.createVarchar(80))) + .build(); + + public ShowTriggersCommand() { + super(PlanType.SHOW_TRIGGERS_COMMAND); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowTriggersCommand(this, context); + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + List> rowSet = Lists.newArrayList(); + return new ShowResultSet(META_DATA, rowSet); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 7aa935df8bfe62d..fa5b9486ed5fc95 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -84,6 +84,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTabletsBelongCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTrashCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowTriggersCommand; import org.apache.doris.nereids.trees.plans.commands.ShowVariablesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowWhiteListCommand; @@ -333,6 +334,10 @@ default R visitShowPluginsCommand(ShowPluginsCommand showPluginsCommand, C conte return visitCommand(showPluginsCommand, context); } + default R visitShowTriggersCommand(ShowTriggersCommand showTriggersCommand, C context) { + return visitCommand(showTriggersCommand, context); + } + default R visitShowRepositoriesCommand(ShowRepositoriesCommand showRepositoriesCommand, C context) { return visitCommand(showRepositoriesCommand, context); } diff --git a/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy b/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy index 7721130d5586f0d..e994e22ec997096 100644 --- a/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy +++ b/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy @@ -25,4 +25,6 @@ suite("test_show_commands_nereids") { checkNereidsExecute("""show frontends;""") checkNereidsExecute("""show backends;""") checkNereidsExecute("""show whitelist;""") + checkNereidsExecute("""show triggers;""") + } From 11276d11fc4f3eb64db0bed3218701f44cdf6588 Mon Sep 17 00:00:00 2001 From: Jerry Hu Date: Fri, 22 Nov 2024 11:47:16 +0800 Subject: [PATCH 034/110] [opt](scan) Release instances of Segment to avoid consuming a large amount of memory in ParallelScannerBuilder (#44189) For wide tables, a loaded Segment will consume a large amount of memory, so it is necessary to release instances of `Segment` as soon as possible. --- be/src/olap/parallel_scanner_builder.cpp | 20 +++++++++++--------- be/src/olap/parallel_scanner_builder.h | 2 +- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/be/src/olap/parallel_scanner_builder.cpp b/be/src/olap/parallel_scanner_builder.cpp index 33e2762aa446219..88c69ab5c9a584a 100644 --- a/be/src/olap/parallel_scanner_builder.cpp +++ b/be/src/olap/parallel_scanner_builder.cpp @@ -24,6 +24,7 @@ #include "cloud/config.h" #include "common/status.h" #include "olap/rowset/beta_rowset.h" +#include "olap/segment_loader.h" #include "pipeline/exec/olap_scan_operator.h" #include "vec/exec/scan/new_olap_scanner.h" @@ -63,21 +64,18 @@ Status ParallelScannerBuilder::_build_scanners_by_rowid(std::list& auto rowset = reader->rowset(); const auto rowset_id = rowset->rowset_id(); - DCHECK(_segment_cache_handles.contains(rowset_id)); - auto& segment_cache_handle = _segment_cache_handles[rowset_id]; + const auto& segments_rows = _all_segments_rows[rowset_id]; if (rowset->num_rows() == 0) { continue; } - const auto& segments = segment_cache_handle.get_segments(); int segment_start = 0; auto split = RowSetSplits(reader->clone()); - for (size_t i = 0; i != segments.size(); ++i) { - const auto& segment = segments[i]; + for (size_t i = 0; i != segments_rows.size(); ++i) { + const size_t rows_of_segment = segments_rows[i]; RowRanges row_ranges; - const size_t rows_of_segment = segment->num_rows(); int64_t offset_in_segment = 0; // try to split large segments into RowRanges @@ -125,7 +123,7 @@ Status ParallelScannerBuilder::_build_scanners_by_rowid(std::list& // The non-empty `row_ranges` means there are some rows left in this segment not added into `split`. if (!row_ranges.is_empty()) { DCHECK_GT(rows_collected, 0); - DCHECK_EQ(row_ranges.to(), segment->num_rows()); + DCHECK_EQ(row_ranges.to(), rows_of_segment); split.segment_row_ranges.emplace_back(std::move(row_ranges)); } } @@ -133,7 +131,7 @@ Status ParallelScannerBuilder::_build_scanners_by_rowid(std::list& DCHECK_LE(rows_collected, _rows_per_scanner); if (rows_collected > 0) { split.segment_offsets.first = segment_start; - split.segment_offsets.second = segments.size(); + split.segment_offsets.second = segments_rows.size(); DCHECK_GT(split.segment_offsets.second, split.segment_offsets.first); DCHECK_EQ(split.segment_row_ranges.size(), split.segment_offsets.second - split.segment_offsets.first); @@ -181,11 +179,15 @@ Status ParallelScannerBuilder::_load() { auto rowset = rs_split.rs_reader->rowset(); RETURN_IF_ERROR(rowset->load()); const auto rowset_id = rowset->rowset_id(); - auto& segment_cache_handle = _segment_cache_handles[rowset_id]; + SegmentCacheHandle segment_cache_handle; RETURN_IF_ERROR(SegmentLoader::instance()->load_segments( std::dynamic_pointer_cast(rowset), &segment_cache_handle, enable_segment_cache, false)); + + for (const auto& segment : segment_cache_handle.get_segments()) { + _all_segments_rows[rowset_id].emplace_back(segment->num_rows()); + } _total_rows += rowset->num_rows(); } } diff --git a/be/src/olap/parallel_scanner_builder.h b/be/src/olap/parallel_scanner_builder.h index 934d769ed59aa09..7c6b5648e89e043 100644 --- a/be/src/olap/parallel_scanner_builder.h +++ b/be/src/olap/parallel_scanner_builder.h @@ -83,7 +83,7 @@ class ParallelScannerBuilder { size_t _rows_per_scanner {_min_rows_per_scanner}; - std::map _segment_cache_handles; + std::map> _all_segments_rows; std::shared_ptr _scanner_profile; RuntimeState* _state; From 82f33c2ce77cb6bcc23642e692553d6ec702b1ba Mon Sep 17 00:00:00 2001 From: wuwenchi Date: Fri, 22 Nov 2024 12:11:26 +0800 Subject: [PATCH 035/110] [feat](iceberg)Supports using `rest` type catalog to read tables in unity catalog (#43525) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What problem does this PR solve? 1. We now support using the `rest` type catalog to read tables in the unity catalog (https://github.com/unitycatalog/unitycatalog). 2. When reading the parquet file on the be side, we find the corresponding column name based on the column id, which naturally supports the column rename function. example: ``` CREATE CATALOG `uc3` PROPERTIES ( "type" = "iceberg", "iceberg.catalog.type" = "rest", "uri" = "http://127.0.0.1:8080/api/2.1/unity-catalog/iceberg", "external_catalog.name" = "unity" --- catalog name in unity catalog ); ``` --- be/src/io/fs/file_system.cpp | 34 ++++++--- be/src/io/fs/file_system.h | 2 +- be/src/io/fs/local_file_system.cpp | 50 ++++++++++++++ be/src/io/fs/local_file_system.h | 6 +- be/src/io/fs/remote_file_system.cpp | 10 ++- be/src/io/fs/remote_file_system.h | 8 ++- be/src/io/fs/s3_file_system.h | 7 +- .../vec/exec/format/parquet/schema_desc.cpp | 17 +++++ be/src/vec/exec/format/parquet/schema_desc.h | 8 +++ .../exec/format/parquet/vparquet_reader.cpp | 6 +- .../vec/exec/format/parquet/vparquet_reader.h | 2 +- .../vec/exec/format/table/iceberg_reader.cpp | 48 ++++--------- be/src/vec/exec/format/table/iceberg_reader.h | 2 +- be/test/io/fs/local_file_system_test.cpp | 50 ++++++++++++++ .../doris/datasource/ExternalCatalog.java | 6 ++ .../datasource/hive/HMSExternalCatalog.java | 20 ++++-- .../iceberg/IcebergExternalCatalog.java | 3 +- .../iceberg/IcebergMetadataCache.java | 14 ++-- .../iceberg/IcebergMetadataOps.java | 69 ++++++++++++++----- .../iceberg/IcebergRestExternalCatalog.java | 2 - .../iceberg/IcebergTransaction.java | 3 +- .../ExternalMetadataOperations.java | 4 +- .../operations/ExternalMetadataOps.java | 4 ++ .../iceberg_read_unitycatalog_table.out | 40 +++++++++++ .../iceberg_read_unitycatalog_table.groovy | 62 +++++++++++++++++ 25 files changed, 376 insertions(+), 101 deletions(-) create mode 100644 regression-test/data/external_table_p0/iceberg/iceberg_read_unitycatalog_table.out create mode 100644 regression-test/suites/external_table_p0/iceberg/iceberg_read_unitycatalog_table.groovy diff --git a/be/src/io/fs/file_system.cpp b/be/src/io/fs/file_system.cpp index 3579a5323d92172..e6b5ef7df1a8f5d 100644 --- a/be/src/io/fs/file_system.cpp +++ b/be/src/io/fs/file_system.cpp @@ -25,58 +25,70 @@ namespace io { Status FileSystem::create_file(const Path& file, FileWriterPtr* writer, const FileWriterOptions* opts) { - auto path = absolute_path(file); + Path path; + RETURN_IF_ERROR(absolute_path(file, path)); FILESYSTEM_M(create_file_impl(path, writer, opts)); } Status FileSystem::open_file(const Path& file, FileReaderSPtr* reader, const FileReaderOptions* opts) { - auto path = absolute_path(file); + Path path; + RETURN_IF_ERROR(absolute_path(file, path)); FILESYSTEM_M(open_file_impl(path, reader, opts)); } Status FileSystem::create_directory(const Path& dir, bool failed_if_exists) { - auto path = absolute_path(dir); + Path path; + RETURN_IF_ERROR(absolute_path(dir, path)); FILESYSTEM_M(create_directory_impl(path, failed_if_exists)); } Status FileSystem::delete_file(const Path& file) { - auto path = absolute_path(file); + Path path; + RETURN_IF_ERROR(absolute_path(file, path)); FILESYSTEM_M(delete_file_impl(path)); } Status FileSystem::delete_directory(const Path& dir) { - auto path = absolute_path(dir); + Path path; + RETURN_IF_ERROR(absolute_path(dir, path)); FILESYSTEM_M(delete_directory_impl(path)); } Status FileSystem::batch_delete(const std::vector& files) { std::vector abs_files; for (auto& file : files) { - abs_files.push_back(absolute_path(file)); + Path abs_file; + RETURN_IF_ERROR(absolute_path(file, abs_file)); + abs_files.push_back(abs_file); } FILESYSTEM_M(batch_delete_impl(abs_files)); } Status FileSystem::exists(const Path& path, bool* res) const { - auto fs_path = absolute_path(path); + Path fs_path; + RETURN_IF_ERROR(absolute_path(path, fs_path)); FILESYSTEM_M(exists_impl(fs_path, res)); } Status FileSystem::file_size(const Path& file, int64_t* file_size) const { - auto path = absolute_path(file); + Path path; + RETURN_IF_ERROR(absolute_path(file, path)); FILESYSTEM_M(file_size_impl(path, file_size)); } Status FileSystem::list(const Path& dir, bool only_file, std::vector* files, bool* exists) { - auto path = absolute_path(dir); + Path path; + RETURN_IF_ERROR(absolute_path(dir, path)); FILESYSTEM_M(list_impl(path, only_file, files, exists)); } Status FileSystem::rename(const Path& orig_name, const Path& new_name) { - auto orig_path = absolute_path(orig_name); - auto new_path = absolute_path(new_name); + Path orig_path; + RETURN_IF_ERROR(absolute_path(orig_name, orig_path)); + Path new_path; + RETURN_IF_ERROR(absolute_path(new_name, new_path)); FILESYSTEM_M(rename_impl(orig_path, new_path)); } diff --git a/be/src/io/fs/file_system.h b/be/src/io/fs/file_system.h index a8ccc8756bb60a7..6baf07917d35cb4 100644 --- a/be/src/io/fs/file_system.h +++ b/be/src/io/fs/file_system.h @@ -163,7 +163,7 @@ class FileSystem { // FIMXE(plat1ko): The implementation and semantics of this function are not completely // consistent, which is confused. - virtual Path absolute_path(const Path& path) const = 0; + virtual Status absolute_path(const Path& path, Path& abs_path) const = 0; FileSystem(std::string id, FileSystemType type) : _id(std::move(id)), _type(type) {} diff --git a/be/src/io/fs/local_file_system.cpp b/be/src/io/fs/local_file_system.cpp index 0107ed57dc8fb16..9270d919a37519d 100644 --- a/be/src/io/fs/local_file_system.cpp +++ b/be/src/io/fs/local_file_system.cpp @@ -471,4 +471,54 @@ Status LocalFileSystem::permission_impl(const Path& file, std::filesystem::perms return Status::OK(); } +Status LocalFileSystem::convert_to_abs_path(const Path& input_path_str, Path& abs_path) { + // valid path include: + // 1. abc/def will return abc/def + // 2. /abc/def will return /abc/def + // 3. file:/abc/def will return /abc/def + // 4. file:///abc/def will return /abc/def + std::string path_str = input_path_str; + size_t slash = path_str.find('/'); + if (slash == 0) { + abs_path = input_path_str; + return Status::OK(); + } + + // Initialize scheme and authority + std::string scheme; + size_t start = 0; + + // Parse URI scheme + size_t colon = path_str.find(':'); + if (colon != std::string::npos && (slash == std::string::npos || colon < slash)) { + // Has a scheme + scheme = path_str.substr(0, colon); + if (scheme != "file") { + return Status::InternalError( + "Only supports `file` type scheme, like 'file:///path', 'file:/path'."); + } + start = colon + 1; + } + + // Parse URI authority, if any + if (path_str.compare(start, 2, "//") == 0 && path_str.length() - start > 2) { + // Has authority + // such as : path_str = "file://authority/abc/def" + // and now : start = 5 + size_t next_slash = path_str.find('/', start + 2); + // now : next_slash = 16 + if (next_slash == std::string::npos) { + return Status::InternalError( + "This input string only has authority, but has no path information"); + } + // We will skit authority + // now : start = 16 + start = next_slash; + } + + // URI path is the rest of the string + abs_path = path_str.substr(start); + return Status::OK(); +} + } // namespace doris::io diff --git a/be/src/io/fs/local_file_system.h b/be/src/io/fs/local_file_system.h index c6295b0bae1d6c5..4540df47c16d810 100644 --- a/be/src/io/fs/local_file_system.h +++ b/be/src/io/fs/local_file_system.h @@ -34,6 +34,8 @@ class LocalFileSystem final : public FileSystem { public: ~LocalFileSystem() override; + static Status convert_to_abs_path(const Path& path, Path& abs_path); + /// hard link dest file to src file Status link_file(const Path& src, const Path& dest); @@ -104,7 +106,9 @@ class LocalFileSystem final : public FileSystem { // `LocalFileSystem` always use absolute path as arguments // FIXME(plat1ko): Eliminate this method - Path absolute_path(const Path& path) const override { return path; } + Status absolute_path(const Path& path, Path& abs_path) const override { + return convert_to_abs_path(path, abs_path); + } friend const std::shared_ptr& global_local_filesystem(); }; diff --git a/be/src/io/fs/remote_file_system.cpp b/be/src/io/fs/remote_file_system.cpp index 2b6af2af046afca..fd793f60cdc5d8d 100644 --- a/be/src/io/fs/remote_file_system.cpp +++ b/be/src/io/fs/remote_file_system.cpp @@ -29,7 +29,8 @@ namespace doris::io { Status RemoteFileSystem::upload(const Path& local_file, const Path& dest_file) { - auto dest_path = absolute_path(dest_file); + Path dest_path; + RETURN_IF_ERROR(absolute_path(dest_file, dest_path)); FILESYSTEM_M(upload_impl(local_file, dest_path)); } @@ -37,13 +38,16 @@ Status RemoteFileSystem::batch_upload(const std::vector& local_files, const std::vector& remote_files) { std::vector remote_paths; for (auto& path : remote_files) { - remote_paths.push_back(absolute_path(path)); + Path abs_path; + RETURN_IF_ERROR(absolute_path(path, abs_path)); + remote_paths.push_back(abs_path); } FILESYSTEM_M(batch_upload_impl(local_files, remote_paths)); } Status RemoteFileSystem::download(const Path& remote_file, const Path& local) { - auto remote_path = absolute_path(remote_file); + Path remote_path; + RETURN_IF_ERROR(absolute_path(remote_file, remote_path)); FILESYSTEM_M(download_impl(remote_path, local)); } diff --git a/be/src/io/fs/remote_file_system.h b/be/src/io/fs/remote_file_system.h index e9472140ab7b08e..de0a1b71519a922 100644 --- a/be/src/io/fs/remote_file_system.h +++ b/be/src/io/fs/remote_file_system.h @@ -64,11 +64,13 @@ class RemoteFileSystem : public FileSystem { virtual Status open_file_internal(const Path& file, FileReaderSPtr* reader, const FileReaderOptions& opts) = 0; - Path absolute_path(const Path& path) const override { + Status absolute_path(const Path& path, Path& abs_path) const override { if (path.is_absolute()) { - return path; + abs_path = path; + } else { + abs_path = _root_path / path; } - return _root_path / path; + return Status::OK(); } Path _root_path; diff --git a/be/src/io/fs/s3_file_system.h b/be/src/io/fs/s3_file_system.h index d1e8b5b6e31a619..61967a63e443796 100644 --- a/be/src/io/fs/s3_file_system.h +++ b/be/src/io/fs/s3_file_system.h @@ -113,16 +113,17 @@ class S3FileSystem final : public RemoteFileSystem { const std::vector& remote_files) override; Status download_impl(const Path& remote_file, const Path& local_file) override; - Path absolute_path(const Path& path) const override { + Status absolute_path(const Path& path, Path& abs_path) const override { if (path.string().find("://") != std::string::npos) { // the path is with schema, which means this is a full path like: // s3://bucket/path/to/file.txt // so no need to concat with prefix - return path; + abs_path = path; } else { // path with no schema - return _root_path / path; + abs_path = _root_path / path; } + return Status::OK(); } private: diff --git a/be/src/vec/exec/format/parquet/schema_desc.cpp b/be/src/vec/exec/format/parquet/schema_desc.cpp index de8797062640287..1eae65b1a4db19b 100644 --- a/be/src/vec/exec/format/parquet/schema_desc.cpp +++ b/be/src/vec/exec/format/parquet/schema_desc.cpp @@ -137,6 +137,9 @@ Status FieldDescriptor::parse_from_thrift(const std::vectorsecond.data()}; +} + Status FieldDescriptor::parse_node_field(const std::vector& t_schemas, size_t curr_pos, FieldSchema* node_field) { if (curr_pos >= t_schemas.size()) { @@ -172,6 +183,7 @@ Status FieldDescriptor::parse_node_field(const std::vectortype.add_sub_type(child->type); node_field->is_nullable = false; _next_schema_pos = curr_pos + 1; + node_field->field_id = t_schema.__isset.field_id ? t_schema.field_id : -1; } else { bool is_optional = is_optional_node(t_schema); if (is_optional) { @@ -194,6 +206,7 @@ void FieldDescriptor::parse_physical_field(const tparquet::SchemaElement& physic auto type = get_doris_type(physical_schema); physical_field->type = type.first; physical_field->is_type_compatibility = type.second; + physical_field->field_id = physical_schema.__isset.field_id ? physical_schema.field_id : -1; } std::pair FieldDescriptor::get_doris_type( @@ -465,6 +478,7 @@ Status FieldDescriptor::parse_group_field(const std::vectortype.type = TYPE_ARRAY; group_field->type.add_sub_type(struct_field->type); group_field->is_nullable = false; + group_field->field_id = group_schema.__isset.field_id ? group_schema.field_id : -1; } else { RETURN_IF_ERROR(parse_struct_field(t_schemas, curr_pos, group_field)); } @@ -533,6 +547,7 @@ Status FieldDescriptor::parse_list_field(const std::vectortype.type = TYPE_ARRAY; list_field->type.add_sub_type(list_field->children[0].type); list_field->is_nullable = is_optional; + list_field->field_id = first_level.__isset.field_id ? first_level.field_id : -1; return Status::OK(); } @@ -597,6 +612,7 @@ Status FieldDescriptor::parse_map_field(const std::vectortype.add_sub_type(map_kv_field->type.children[0]); map_field->type.add_sub_type(map_kv_field->type.children[1]); map_field->is_nullable = is_optional; + map_field->field_id = map_schema.__isset.field_id ? map_schema.field_id : -1; return Status::OK(); } @@ -619,6 +635,7 @@ Status FieldDescriptor::parse_struct_field(const std::vectorname = to_lower(struct_schema.name); struct_field->is_nullable = is_optional; struct_field->type.type = TYPE_STRUCT; + struct_field->field_id = struct_schema.__isset.field_id ? struct_schema.field_id : -1; for (int i = 0; i < num_children; ++i) { struct_field->type.add_sub_type(struct_field->children[i].type, struct_field->children[i].name); diff --git a/be/src/vec/exec/format/parquet/schema_desc.h b/be/src/vec/exec/format/parquet/schema_desc.h index ca726ef1b575900..2593da837c3da66 100644 --- a/be/src/vec/exec/format/parquet/schema_desc.h +++ b/be/src/vec/exec/format/parquet/schema_desc.h @@ -28,6 +28,7 @@ #include "common/status.h" #include "runtime/types.h" +#include "util/slice.h" namespace doris::vectorized { @@ -56,6 +57,8 @@ struct FieldSchema { ~FieldSchema() = default; FieldSchema(const FieldSchema& fieldSchema) = default; std::string debug_string() const; + + int32_t field_id; }; class FieldDescriptor { @@ -68,6 +71,7 @@ class FieldDescriptor { std::unordered_map _name_to_field; // Used in from_thrift, marking the next schema position that should be parsed size_t _next_schema_pos; + std::unordered_map _field_id_name_mapping; void parse_physical_field(const tparquet::SchemaElement& physical_schema, bool is_nullable, FieldSchema* physical_field); @@ -128,6 +132,10 @@ class FieldDescriptor { std::string debug_string() const; int32_t size() const { return _fields.size(); } + + bool has_parquet_field_id() const { return _field_id_name_mapping.size() > 0; } + + const doris::Slice get_column_name_from_field_id(int32_t id) const; }; } // namespace doris::vectorized diff --git a/be/src/vec/exec/format/parquet/vparquet_reader.cpp b/be/src/vec/exec/format/parquet/vparquet_reader.cpp index 1a3ae2f885e4a06..47209dbb332cb8d 100644 --- a/be/src/vec/exec/format/parquet/vparquet_reader.cpp +++ b/be/src/vec/exec/format/parquet/vparquet_reader.cpp @@ -253,10 +253,8 @@ Status ParquetReader::_open_file() { return Status::OK(); } -// Get iceberg col id to col name map stored in parquet metadata key values. -// This is for iceberg schema evolution. -std::vector ParquetReader::get_metadata_key_values() { - return _t_metadata->key_value_metadata; +const FieldDescriptor ParquetReader::get_file_metadata_schema() { + return _file_metadata->schema(); } Status ParquetReader::open() { diff --git a/be/src/vec/exec/format/parquet/vparquet_reader.h b/be/src/vec/exec/format/parquet/vparquet_reader.h index a7768c8bd6c4f85..b2aa156a79d0887 100644 --- a/be/src/vec/exec/format/parquet/vparquet_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_reader.h @@ -148,7 +148,7 @@ class ParquetReader : public GenericReader { partition_columns, const std::unordered_map& missing_columns) override; - std::vector get_metadata_key_values(); + const FieldDescriptor get_file_metadata_schema(); void set_table_to_file_col_map(std::unordered_map& map) { _table_col_to_file_col = map; } diff --git a/be/src/vec/exec/format/table/iceberg_reader.cpp b/be/src/vec/exec/format/table/iceberg_reader.cpp index 295a3a405441b61..8f130ca6002d5d4 100644 --- a/be/src/vec/exec/format/table/iceberg_reader.cpp +++ b/be/src/vec/exec/format/table/iceberg_reader.cpp @@ -53,6 +53,7 @@ #include "vec/exec/format/format_common.h" #include "vec/exec/format/generic_reader.h" #include "vec/exec/format/orc/vorc_reader.h" +#include "vec/exec/format/parquet/schema_desc.h" #include "vec/exec/format/table/table_format_reader.h" namespace cctz { @@ -546,8 +547,8 @@ Status IcebergParquetReader::init_reader( _col_id_name_map = col_id_name_map; _file_col_names = file_col_names; _colname_to_value_range = colname_to_value_range; - auto parquet_meta_kv = parquet_reader->get_metadata_key_values(); - RETURN_IF_ERROR(_gen_col_name_maps(parquet_meta_kv)); + FieldDescriptor field_desc = parquet_reader->get_file_metadata_schema(); + RETURN_IF_ERROR(_gen_col_name_maps(field_desc)); _gen_file_col_names(); _gen_new_colname_to_value_range(); parquet_reader->set_table_to_file_col_map(_table_col_to_file_col); @@ -672,39 +673,20 @@ Status IcebergOrcReader::_read_position_delete_file(const TFileRangeDesc* delete * 1. col1_new -> col1 * 2. col1 -> col1_new */ -Status IcebergParquetReader::_gen_col_name_maps(std::vector parquet_meta_kv) { - for (int i = 0; i < parquet_meta_kv.size(); ++i) { - tparquet::KeyValue kv = parquet_meta_kv[i]; - if (kv.key == "iceberg.schema") { - _has_iceberg_schema = true; - std::string schema = kv.value; - rapidjson::Document json; - json.Parse(schema.c_str()); - - if (json.HasMember("fields")) { - rapidjson::Value& fields = json["fields"]; - if (fields.IsArray()) { - for (int j = 0; j < fields.Size(); j++) { - rapidjson::Value& e = fields[j]; - rapidjson::Value& id = e["id"]; - rapidjson::Value& name = e["name"]; - std::string name_string = name.GetString(); - transform(name_string.begin(), name_string.end(), name_string.begin(), - ::tolower); - auto iter = _col_id_name_map.find(id.GetInt()); - if (iter != _col_id_name_map.end()) { - _table_col_to_file_col.emplace(iter->second, name_string); - _file_col_to_table_col.emplace(name_string, iter->second); - if (name_string != iter->second) { - _has_schema_change = true; - } - } else { - _has_schema_change = true; - } - } +Status IcebergParquetReader::_gen_col_name_maps(const FieldDescriptor& field_desc) { + if (field_desc.has_parquet_field_id()) { + for (const auto& pair : _col_id_name_map) { + auto name_slice = field_desc.get_column_name_from_field_id(pair.first); + if (name_slice.get_size() == 0) { + _has_schema_change = true; + } else { + auto name_string = name_slice.to_string(); + _table_col_to_file_col.emplace(pair.second, name_string); + _file_col_to_table_col.emplace(name_string, pair.second); + if (name_string != pair.second) { + _has_schema_change = true; } } - break; } } return Status::OK(); diff --git a/be/src/vec/exec/format/table/iceberg_reader.h b/be/src/vec/exec/format/table/iceberg_reader.h index 04f64aad518a9fb..2e240f465b6a2cd 100644 --- a/be/src/vec/exec/format/table/iceberg_reader.h +++ b/be/src/vec/exec/format/table/iceberg_reader.h @@ -218,7 +218,7 @@ class IcebergParquetReader final : public IcebergTableReader { parquet_reader->set_delete_rows(&_iceberg_delete_rows); } - Status _gen_col_name_maps(std::vector parquet_meta_kv); + Status _gen_col_name_maps(const FieldDescriptor& field_desc); protected: std::unique_ptr _create_equality_reader( diff --git a/be/test/io/fs/local_file_system_test.cpp b/be/test/io/fs/local_file_system_test.cpp index 0fd18445beac4c8..c930ba72eabf867 100644 --- a/be/test/io/fs/local_file_system_test.cpp +++ b/be/test/io/fs/local_file_system_test.cpp @@ -417,4 +417,54 @@ TEST_F(LocalFileSystemTest, TestGlob) { EXPECT_TRUE(io::global_local_filesystem()->delete_directory(path).ok()); } +TEST_F(LocalFileSystemTest, TestConvertToAbsPath) { + io::Path abs_path; + Status st; + + // suppurt path: + st = doris::io::LocalFileSystem::convert_to_abs_path("/abc/def", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/abc/def", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file:/def/hij", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/def/hij", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file://host:80/hij/abc", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/hij/abc", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file://host/abc/def", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/abc/def", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file:///def", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/def", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file:///", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file://auth/", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("abc", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("abc", abs_path); + + // not support path: + st = doris::io::LocalFileSystem::convert_to_abs_path("file://auth", abs_path); + ASSERT_TRUE(!st.ok()); + + st = doris::io::LocalFileSystem::convert_to_abs_path("fileee:/abc", abs_path); + ASSERT_TRUE(!st.ok()); + + st = doris::io::LocalFileSystem::convert_to_abs_path("hdfs:///abc", abs_path); + ASSERT_TRUE(!st.ok()); + + st = doris::io::LocalFileSystem::convert_to_abs_path("hdfs:/abc", abs_path); + ASSERT_TRUE(!st.ok()); +} } // namespace doris diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java index 345254108345068..091ee347b101551 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java @@ -39,6 +39,7 @@ import org.apache.doris.common.Version; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; +import org.apache.doris.common.security.authentication.PreExecutionAuthenticator; import org.apache.doris.common.util.Util; import org.apache.doris.datasource.es.EsExternalDatabase; import org.apache.doris.datasource.hive.HMSExternalCatalog; @@ -149,6 +150,7 @@ public abstract class ExternalCatalog protected Optional useMetaCache = Optional.empty(); protected MetaCache> metaCache; + protected PreExecutionAuthenticator preExecutionAuthenticator; public ExternalCatalog() { } @@ -934,4 +936,8 @@ public void setAutoAnalyzePolicy(String dbName, String tableName, String policy) tableAutoAnalyzePolicy.put(key, policy); } } + + public PreExecutionAuthenticator getPreExecutionAuthenticator() { + return preExecutionAuthenticator; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java index 20b9482041df02f..85b999f11110478 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java @@ -27,6 +27,7 @@ import org.apache.doris.common.ThreadPoolManager; import org.apache.doris.common.security.authentication.AuthenticationConfig; import org.apache.doris.common.security.authentication.HadoopAuthenticator; +import org.apache.doris.common.security.authentication.PreExecutionAuthenticator; import org.apache.doris.common.util.Util; import org.apache.doris.datasource.CatalogProperty; import org.apache.doris.datasource.ExternalCatalog; @@ -34,6 +35,7 @@ import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.InitCatalogLog; import org.apache.doris.datasource.SessionContext; +import org.apache.doris.datasource.iceberg.IcebergMetadataOps; import org.apache.doris.datasource.iceberg.IcebergUtils; import org.apache.doris.datasource.jdbc.client.JdbcClientConfig; import org.apache.doris.datasource.operations.ExternalMetadataOperations; @@ -88,7 +90,7 @@ public class HMSExternalCatalog extends ExternalCatalog { private boolean enableHmsEventsIncrementalSync = false; //for "type" = "hms" , but is iceberg table. - private HiveCatalog icebergHiveCatalog; + private IcebergMetadataOps icebergMetadataOps; @VisibleForTesting public HMSExternalCatalog() { @@ -168,6 +170,7 @@ public void checkProperties() throws DdlException { @Override protected void initLocalObjectsImpl() { + preExecutionAuthenticator = new PreExecutionAuthenticator(); if (authenticator == null) { AuthenticationConfig config = AuthenticationConfig.getKerberosConfig(getConfiguration()); authenticator = HadoopAuthenticator.getHadoopAuthenticator(config); @@ -199,8 +202,6 @@ protected void initLocalObjectsImpl() { transactionManager = TransactionManagerFactory.createHiveTransactionManager(hiveOps, fileSystemProvider, fileSystemExecutor); metadataOps = hiveOps; - - icebergHiveCatalog = IcebergUtils.createIcebergHiveCatalog(this, getName()); } @Override @@ -337,10 +338,6 @@ public boolean isEnableHmsEventsIncrementalSync() { return enableHmsEventsIncrementalSync; } - public HiveCatalog getIcebergHiveCatalog() { - return icebergHiveCatalog; - } - /** * Enum for meta tables in hive catalog. * eg: tbl$partitions @@ -393,5 +390,14 @@ public TableValuedFunctionRef createFunctionRef(String ctlName, String dbName, S } } } + + public IcebergMetadataOps getIcebergMetadataOps() { + makeSureInitialized(); + if (icebergMetadataOps == null) { + HiveCatalog icebergHiveCatalog = IcebergUtils.createIcebergHiveCatalog(this, getName()); + icebergMetadataOps = ExternalMetadataOperations.newIcebergMetadataOps(this, icebergHiveCatalog); + } + return icebergMetadataOps; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java index d8dfd1c128f162e..0fa69825a01ef49 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java @@ -40,11 +40,10 @@ public abstract class IcebergExternalCatalog extends ExternalCatalog { public static final String ICEBERG_HADOOP = "hadoop"; public static final String ICEBERG_GLUE = "glue"; public static final String ICEBERG_DLF = "dlf"; + public static final String EXTERNAL_CATALOG_NAME = "external_catalog.name"; protected String icebergCatalogType; protected Catalog catalog; - protected PreExecutionAuthenticator preExecutionAuthenticator; - public IcebergExternalCatalog(long catalogId, String name, String comment) { super(catalogId, name, InitCatalogLog.Type.ICEBERG, comment); } 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 c1ac2a79754b796..ad347ca78f2a4f3 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 @@ -36,8 +36,6 @@ import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; import org.jetbrains.annotations.NotNull; import java.util.HashMap; @@ -104,18 +102,16 @@ private List loadSnapshots(IcebergMetadataCacheKey key) { @NotNull private Table loadTable(IcebergMetadataCacheKey key) { - Catalog icebergCatalog; + IcebergMetadataOps ops; if (key.catalog instanceof HMSExternalCatalog) { - icebergCatalog = ((HMSExternalCatalog) key.catalog).getIcebergHiveCatalog(); + ops = ((HMSExternalCatalog) key.catalog).getIcebergMetadataOps(); } else if (key.catalog instanceof IcebergExternalCatalog) { - icebergCatalog = ((IcebergExternalCatalog) key.catalog).getCatalog(); + ops = (IcebergMetadataOps) (((IcebergExternalCatalog) key.catalog).getMetadataOps()); } else { throw new RuntimeException("Only support 'hms' and 'iceberg' type for iceberg table"); } - Table icebergTable = HiveMetaStoreClientHelper.ugiDoAs(((ExternalCatalog) key.catalog).getConfiguration(), - () -> icebergCatalog.loadTable(TableIdentifier.of(key.dbName, key.tableName))); - initIcebergTableFileIO(icebergTable, key.catalog.getProperties()); - return icebergTable; + return HiveMetaStoreClientHelper.ugiDoAs(((ExternalCatalog) key.catalog).getConfiguration(), + () -> ops.loadTable(key.dbName, key.tableName)); } public void invalidateCatalogCache(long catalogId) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java index 59729ddb47a6a42..970814b7acdc85e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java @@ -36,6 +36,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; @@ -46,29 +47,40 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; public class IcebergMetadataOps implements ExternalMetadataOps { private static final Logger LOG = LogManager.getLogger(IcebergMetadataOps.class); protected Catalog catalog; - protected IcebergExternalCatalog dorisCatalog; + protected ExternalCatalog dorisCatalog; protected SupportsNamespaces nsCatalog; private PreExecutionAuthenticator preExecutionAuthenticator; + // Generally, there should be only two levels under the catalog, namely .
, + // but the REST type catalog is obtained from an external server, + // and the level provided by the external server may be three levels, ..
. + // Therefore, if the external server provides a catalog, + // the catalog needs to be recorded here to ensure semantic consistency. + private Optional externalCatalogName = Optional.empty(); - public IcebergMetadataOps(IcebergExternalCatalog dorisCatalog, Catalog catalog) { + public IcebergMetadataOps(ExternalCatalog dorisCatalog, Catalog catalog) { this.dorisCatalog = dorisCatalog; this.catalog = catalog; nsCatalog = (SupportsNamespaces) catalog; - this.preExecutionAuthenticator = dorisCatalog.preExecutionAuthenticator; + this.preExecutionAuthenticator = dorisCatalog.getPreExecutionAuthenticator(); + if (dorisCatalog.getProperties().containsKey(IcebergExternalCatalog.EXTERNAL_CATALOG_NAME)) { + externalCatalogName = + Optional.of(dorisCatalog.getProperties().get(IcebergExternalCatalog.EXTERNAL_CATALOG_NAME)); + } } public Catalog getCatalog() { return catalog; } - public IcebergExternalCatalog getExternalCatalog() { + public ExternalCatalog getExternalCatalog() { return dorisCatalog; } @@ -78,17 +90,18 @@ public void close() { @Override public boolean tableExist(String dbName, String tblName) { - return catalog.tableExists(TableIdentifier.of(dbName, tblName)); + return catalog.tableExists(getTableIdentifier(dbName, tblName)); } public boolean databaseExist(String dbName) { - return nsCatalog.namespaceExists(Namespace.of(dbName)); + return nsCatalog.namespaceExists(getNamespace(dbName)); } public List listDatabaseNames() { try { - return preExecutionAuthenticator.execute(() -> nsCatalog.listNamespaces().stream() - .map(Namespace::toString) + return preExecutionAuthenticator.execute(() -> nsCatalog.listNamespaces(getNamespace()) + .stream() + .map(n -> n.level(n.length() - 1)) .collect(Collectors.toList())); } catch (Exception e) { throw new RuntimeException("Failed to list database names, error message is: " + e.getMessage()); @@ -98,7 +111,7 @@ public List listDatabaseNames() { @Override public List listTableNames(String dbName) { - List tableIdentifiers = catalog.listTables(Namespace.of(dbName)); + List tableIdentifiers = catalog.listTables(getNamespace(dbName)); return tableIdentifiers.stream().map(TableIdentifier::name).collect(Collectors.toList()); } @@ -128,12 +141,14 @@ private void performCreateDb(CreateDbStmt stmt) throws DdlException { ErrorReport.reportDdlException(ErrorCode.ERR_DB_CREATE_EXISTS, dbName); } } - String icebergCatalogType = dorisCatalog.getIcebergCatalogType(); - if (!properties.isEmpty() && !IcebergExternalCatalog.ICEBERG_HMS.equals(icebergCatalogType)) { - throw new DdlException( + if (!properties.isEmpty() && dorisCatalog instanceof IcebergExternalCatalog) { + String icebergCatalogType = ((IcebergExternalCatalog) dorisCatalog).getIcebergCatalogType(); + if (!IcebergExternalCatalog.ICEBERG_HMS.equals(icebergCatalogType)) { + throw new DdlException( "Not supported: create database with properties for iceberg catalog type: " + icebergCatalogType); + } } - nsCatalog.createNamespace(Namespace.of(dbName), properties); + nsCatalog.createNamespace(getNamespace(dbName), properties); dorisCatalog.onRefreshCache(true); } @@ -159,8 +174,7 @@ private void preformDropDb(DropDbStmt stmt) throws DdlException { ErrorReport.reportDdlException(ErrorCode.ERR_DB_DROP_EXISTS, dbName); } } - SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; - nsCatalog.dropNamespace(Namespace.of(dbName)); + nsCatalog.dropNamespace(getNamespace(dbName)); dorisCatalog.onRefreshCache(true); } @@ -200,7 +214,7 @@ public boolean performCreateTable(CreateTableStmt stmt) throws UserException { Map properties = stmt.getProperties(); properties.put(ExternalCatalog.DORIS_VERSION, ExternalCatalog.DORIS_VERSION_VALUE); PartitionSpec partitionSpec = IcebergUtils.solveIcebergPartitionSpec(stmt.getPartitionDesc(), schema); - catalog.createTable(TableIdentifier.of(dbName, tableName), schema, partitionSpec, properties); + catalog.createTable(getTableIdentifier(dbName, tableName), schema, partitionSpec, properties); db.setUnInitialized(true); return false; } @@ -238,7 +252,7 @@ private void performDropTable(DropTableStmt stmt) throws DdlException { ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_TABLE, tableName, dbName); } } - catalog.dropTable(TableIdentifier.of(dbName, tableName), true); + catalog.dropTable(getTableIdentifier(dbName, tableName), true); db.setUnInitialized(true); } @@ -250,4 +264,25 @@ public void truncateTable(String dbName, String tblName, List partitions public PreExecutionAuthenticator getPreExecutionAuthenticator() { return preExecutionAuthenticator; } + + @Override + public Table loadTable(String dbName, String tblName) { + return catalog.loadTable(getTableIdentifier(dbName, tblName)); + } + + private TableIdentifier getTableIdentifier(String dbName, String tblName) { + return externalCatalogName + .map(s -> TableIdentifier.of(s, dbName, tblName)) + .orElseGet(() -> TableIdentifier.of(dbName, tblName)); + } + + private Namespace getNamespace(String dbName) { + return externalCatalogName + .map(s -> Namespace.of(s, dbName)) + .orElseGet(() -> Namespace.of(dbName)); + } + + private Namespace getNamespace() { + return externalCatalogName.map(Namespace::of).orElseGet(() -> Namespace.empty()); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergRestExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergRestExternalCatalog.java index 908a4fa9e3f2710..b92d2c91f9630e4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergRestExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergRestExternalCatalog.java @@ -26,7 +26,6 @@ import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.aws.AwsClientProperties; -import org.apache.iceberg.aws.s3.S3FileIO; import org.apache.iceberg.aws.s3.S3FileIOProperties; import java.util.HashMap; @@ -71,7 +70,6 @@ private Map convertToRestCatalogProperties() { Map props = catalogProperty.getProperties(); Map restProperties = new HashMap<>(props); - restProperties.put(CatalogProperties.FILE_IO_IMPL, S3FileIO.class.getName()); restProperties.put(CatalogUtil.ICEBERG_CATALOG_TYPE, CatalogUtil.ICEBERG_CATALOG_TYPE_REST); String restUri = props.getOrDefault(CatalogProperties.URI, ""); restProperties.put(CatalogProperties.URI, restUri); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java index 685915025d665e3..d0cca11b0af2bb9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java @@ -22,6 +22,7 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.info.SimpleTableInfo; +import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.iceberg.helper.IcebergWriterHelper; import org.apache.doris.nereids.trees.plans.commands.insert.BaseExternalTableInsertCommandContext; import org.apache.doris.nereids.trees.plans.commands.insert.InsertCommandContext; @@ -140,7 +141,7 @@ public long getUpdateCnt() { private synchronized Table getNativeTable(SimpleTableInfo tableInfo) { Objects.requireNonNull(tableInfo); - IcebergExternalCatalog externalCatalog = ops.getExternalCatalog(); + ExternalCatalog externalCatalog = ops.getExternalCatalog(); return IcebergUtils.getRemoteTable(externalCatalog, tableInfo); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOperations.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOperations.java index 4a2757f918f2946..50166fe83051131 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOperations.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOperations.java @@ -17,9 +17,9 @@ package org.apache.doris.datasource.operations; +import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HiveMetadataOps; -import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; import org.apache.doris.datasource.iceberg.IcebergMetadataOps; import org.apache.doris.datasource.jdbc.client.JdbcClientConfig; @@ -34,7 +34,7 @@ public static HiveMetadataOps newHiveMetadataOps(HiveConf hiveConf, JdbcClientCo return new HiveMetadataOps(hiveConf, jdbcClientConfig, catalog); } - public static IcebergMetadataOps newIcebergMetadataOps(IcebergExternalCatalog dorisCatalog, Catalog catalog) { + public static IcebergMetadataOps newIcebergMetadataOps(ExternalCatalog dorisCatalog, Catalog catalog) { return new IcebergMetadataOps(dorisCatalog, catalog); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java index 0333124b35294c5..e5ed129c679ffe3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java @@ -91,6 +91,10 @@ public interface ExternalMetadataOps { boolean databaseExist(String dbName); + default Object loadTable(String dbName, String tblName) { + throw new UnsupportedOperationException("Load table is not supported."); + } + /** * close the connection, eg, to hms */ diff --git a/regression-test/data/external_table_p0/iceberg/iceberg_read_unitycatalog_table.out b/regression-test/data/external_table_p0/iceberg/iceberg_read_unitycatalog_table.out new file mode 100644 index 000000000000000..42414c3654930c5 --- /dev/null +++ b/regression-test/data/external_table_p0/iceberg/iceberg_read_unitycatalog_table.out @@ -0,0 +1,40 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q1 -- +1 nWYHawtqUw 930 +2 uvOzzthsLV 166 +3 WIAehuXWkv 170 +4 wYCSvnJKTo 709 +5 VsslXsUIDZ 993 +6 ZLsACYYTFy 813 +7 BtDDvLeBpK 52 +8 YISVtrPfGr 8 +9 PBPJHDFjjC 45 +10 qbDuUJzJMO 756 +11 EjqqWoaLJn 712 +12 jpZLMdKXpn 847 +13 acpjQXpJCp 649 +14 nOKqHhRwao 133 +15 kxUUZEUoKv 398 + +-- !q2 -- +7 +8 +9 +10 +11 +12 +13 +14 +15 + +-- !q3 -- +nWYHawtqUw 930 +wYCSvnJKTo 709 +VsslXsUIDZ 993 +ZLsACYYTFy 813 +qbDuUJzJMO 756 +EjqqWoaLJn 712 +jpZLMdKXpn 847 +acpjQXpJCp 649 +kxUUZEUoKv 398 + diff --git a/regression-test/suites/external_table_p0/iceberg/iceberg_read_unitycatalog_table.groovy b/regression-test/suites/external_table_p0/iceberg/iceberg_read_unitycatalog_table.groovy new file mode 100644 index 000000000000000..48b8b6559ca82e1 --- /dev/null +++ b/regression-test/suites/external_table_p0/iceberg/iceberg_read_unitycatalog_table.groovy @@ -0,0 +1,62 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("iceberg_read_unitycatalog_table", "p0,external,doris,external_docker,external_docker_doris") { + + String enabled = context.config.otherConfigs.get("enableIcebergTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable iceberg test.") + return + } + + String catalog_name = "iceberg_read_unitycatalog_table" + String rest_port = context.config.otherConfigs.get("iceberg_rest_uri_port") + String minio_port = context.config.otherConfigs.get("iceberg_minio_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + sql """drop catalog if exists ${catalog_name}""" + sql """ + CREATE CATALOG ${catalog_name} PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='rest', + 'uri' = 'http://${externalEnvIp}:${rest_port}', + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", + "s3.region" = "us-east-1" + );""" + + logger.info("catalog " + catalog_name + " created") + sql """ use ${catalog_name}.test_db """ + String tb = "unitycatalog_marksheet_uniform" + + qt_q1 """ select * from ${tb} order by c1 """ + qt_q2 """ select c1 from ${tb} where c1 > 6 order by c1 """ + qt_q3 """ select c2, c3 from ${tb} where c3 > 200 order by c1 """ + +} + +/* + +spark-sql: + 1. create table marksheet_uniform (c1 int, c2 string, c3 int); + 2. get parquet file from marksheet_uniform; (ref: https://docs.unitycatalog.io/usage/tables/uniform/) + 3. put parquet file to hdfs: hdfs dfs -put hdfs://xxxxx + 4. CALL .system.add_files( + table => '.unitycatalog_db.marksheet_uniform', + source_table => '`parquet`.`hdfs://172.20.32.136:8020/user/doris/preinstalled_data/iceberg_hadoop_warehouse/unitycatalog_db/marksheet_uniform_data/part-00000-5af50cc4-3218-465b-a3a4-eb4fc709421d-c000.snappy.parquet`' + ); +*/ \ No newline at end of file From db02ca1cd96dcabad078bce9878cd1b174927da3 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Fri, 22 Nov 2024 09:55:18 +0530 Subject: [PATCH 036/110] [fix](drop sql) add force in the tosql for drop table and drop database (#43227) 1. What problem was fixed (it's best to include specific error reporting information). How it was fixed. 2. Which behaviors were modified. What was the previous behavior, what is it now, why was it modified, and what possible impacts might there be. 3. What features were added. Why this function was added. 4. Which codes were refactored and why this part of the code was refactored. 5. Which functions were optimized and what is the difference before and after the optimization. --- .../java/org/apache/doris/analysis/DropDbStmt.java | 3 +++ .../org/apache/doris/analysis/DropTableStmt.java | 3 +++ .../org/apache/doris/analysis/DropDbStmtTest.java | 12 +++++++++++- .../org/apache/doris/analysis/DropTableStmtTest.java | 5 +++-- 4 files changed, 20 insertions(+), 3 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java index 2715bd1f6da2f9e..47fdfdce4e23c88 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java @@ -88,6 +88,9 @@ public void analyze(Analyzer analyzer) throws UserException { public String toSql() { StringBuilder stringBuilder = new StringBuilder(); stringBuilder.append("DROP DATABASE ").append("`").append(dbName).append("`"); + if (forceDrop) { + stringBuilder.append(" FORCE"); + } return stringBuilder.toString(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropTableStmt.java index 5e06fce75ee0740..d6a19e81f8e3f47 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropTableStmt.java @@ -100,6 +100,9 @@ public void analyze(Analyzer analyzer) throws UserException { public String toSql() { StringBuilder stringBuilder = new StringBuilder(); stringBuilder.append("DROP TABLE ").append(tableName.toSql()); + if (forceDrop) { + stringBuilder.append(" FORCE"); + } return stringBuilder.toString(); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java index 67b44adc565534e..f14f5113d8feea0 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java @@ -45,7 +45,7 @@ public void setUp() { @Test public void testNormal() throws UserException, AnalysisException { - DropDbStmt stmt = new DropDbStmt(false, new DbName("test", "test"), true); + DropDbStmt stmt = new DropDbStmt(false, new DbName("test", "test"), false); stmt.analyze(analyzer); Assert.assertEquals("test", stmt.getCtlName()); @@ -53,6 +53,16 @@ public void testNormal() throws UserException, AnalysisException { Assert.assertEquals("DROP DATABASE `test`", stmt.toString()); } + @Test + public void testForce() throws UserException, AnalysisException { + DropDbStmt stmt = new DropDbStmt(false, new DbName("test", "test"), true); + + stmt.analyze(analyzer); + Assert.assertEquals("test", stmt.getCtlName()); + Assert.assertEquals("test", stmt.getDbName()); + Assert.assertEquals("DROP DATABASE `test` FORCE", stmt.toString()); + } + @Test(expected = AnalysisException.class) public void testFailed() throws UserException, AnalysisException { DropDbStmt stmt = new DropDbStmt(false, new DbName("", ""), true); diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropTableStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropTableStmtTest.java index da6d5b8d4c44d0a..437e54f58f20e6d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropTableStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropTableStmtTest.java @@ -72,12 +72,13 @@ public void testNormal() throws UserException, AnalysisException { stmt.analyze(analyzer); Assert.assertEquals("db1", stmt.getDbName()); Assert.assertEquals("table1", stmt.getTableName()); - Assert.assertEquals("DROP TABLE `db1`.`table1`", stmt.toString()); + // one with force. + Assert.assertEquals("DROP TABLE `db1`.`table1` FORCE", stmt.toString()); } @Test public void testDefaultNormal() throws UserException, AnalysisException { - DropTableStmt stmt = new DropTableStmt(false, noDbTbl, true); + DropTableStmt stmt = new DropTableStmt(false, noDbTbl, false); stmt.analyze(analyzer); Assert.assertEquals("testDb", stmt.getDbName()); Assert.assertEquals("table1", stmt.getTableName()); From f8191d80e2f5f77f482cf0349215fc116051400d Mon Sep 17 00:00:00 2001 From: feiniaofeiafei Date: Fri, 22 Nov 2024 14:06:22 +0800 Subject: [PATCH 037/110] [Fix](nereids) fix create view and alter view using old parser checking (#43263) Problem: Creating a view on a table with materialized views (MV) results in an error: failed to init view stmt. For example, executing the following SQL: create view v1 as select * from t1 index t_mv_mv. Root Cause: The syntax index t_mv_mv is supported only by the new planner and not by the old planner. During the view creation process, regardless of whether the new or old optimizer is used, the process eventually calls the createView method in Env. In the createView method, the persisted SQL of the view is parsed using the old optimizer's parser. If the parsing fails, an error is thrown. As a result, if the SQL for creating a view includes syntax supported only by the new optimizer, an error occurs. Solution: This PR addresses the issue by removing the parsing check from the createView method in Env. Instead, the check is moved to the respective classes responsible for creating views in the new and old optimizers. --- .../java/org/apache/doris/alter/Alter.java | 5 - .../apache/doris/analysis/BaseViewStmt.java | 26 ++ .../java/org/apache/doris/catalog/Env.java | 7 - .../plans/commands/info/AlterViewInfo.java | 14 +- .../plans/commands/info/BaseViewInfo.java | 4 + .../plans/commands/info/CreateViewInfo.java | 11 +- .../create_view_use_mv.out | 15 + .../test_create_view_nereids.out | 397 ++++++++++++++++++ .../data/ddl_p0/test_create_view_nereids.out | 397 ------------------ .../create_view_use_mv.groovy | 95 +++++ .../test_create_view_nereids.groovy | 0 11 files changed, 548 insertions(+), 423 deletions(-) create mode 100644 regression-test/data/ddl_p0/create_view_nereids/create_view_use_mv.out create mode 100644 regression-test/data/ddl_p0/create_view_nereids/test_create_view_nereids.out delete mode 100644 regression-test/data/ddl_p0/test_create_view_nereids.out create mode 100644 regression-test/suites/ddl_p0/create_view_nereids/create_view_use_mv.groovy rename regression-test/suites/ddl_p0/{ => create_view_nereids}/test_create_view_nereids.groovy (100%) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index ebb194ed6a6262b..a2ed437349850c6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -701,11 +701,6 @@ private void modifyViewDef(Database db, View view, String inlineViewDef, long sq view.writeLockOrDdlException(); try { view.setInlineViewDefWithSqlMode(inlineViewDef, sqlMode); - try { - view.init(); - } catch (UserException e) { - throw new DdlException("failed to init view stmt, reason=" + e.getMessage()); - } view.setNewFullSchema(newFullSchema); String viewName = view.getName(); db.unregisterTable(viewName); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BaseViewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BaseViewStmt.java index e265703128dff70..d33c234af113bf7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BaseViewStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BaseViewStmt.java @@ -21,20 +21,24 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; +import org.apache.doris.common.util.SqlParserUtils; import org.apache.doris.common.util.ToSqlContext; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; +import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.io.StringReader; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -147,6 +151,7 @@ protected void createColumnAndViewDefs(Analyzer analyzer) throws AnalysisExcepti // we don't need the slot id info, so using ToSqlContext to remove it. toSqlContext.setNeedSlotRefId(false); inlineViewDef = viewDefStmt.toSql(); + checkInlineViewDef(); } return; } @@ -161,6 +166,7 @@ protected void createColumnAndViewDefs(Analyzer analyzer) throws AnalysisExcepti // we don't need the slot id info, so using ToSqlContext to remove it. toSqlContext.setNeedSlotRefId(false); inlineViewDef = cloneStmt.toSql(); + checkInlineViewDef(); } } @@ -172,4 +178,24 @@ public void analyze(Analyzer analyzer) throws AnalysisException, UserException { throw new AnalysisException("Not support OUTFILE clause in CREATE VIEW statement"); } } + + private void checkInlineViewDef() throws UserException { + Preconditions.checkNotNull(inlineViewDef); + SqlScanner input = new SqlScanner(new StringReader(inlineViewDef), + ConnectContext.get().getSessionVariable().getSqlMode()); + SqlParser parser = new SqlParser(input); + ParseNode node; + try { + node = SqlParserUtils.getFirstStmt(parser); + } catch (Exception e) { + throw new DdlException( + String.format("Failed to parse view-definition statement of view: %s, stmt is %s, reason is %s", + tableName, inlineViewDef, e.getMessage())); + } + // Make sure the view definition parses to a query statement. + if (!(node instanceof QueryStmt)) { + throw new DdlException(String.format("View definition of %s " + + "is not a query statement", tableName)); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index fa406faf9d34d69..85e041b88b52fb0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -5663,13 +5663,6 @@ public void createView(CreateViewStmt stmt) throws DdlException { newView.setComment(stmt.getComment()); newView.setInlineViewDefWithSqlMode(stmt.getInlineViewDef(), ConnectContext.get().getSessionVariable().getSqlMode()); - // init here in case the stmt string from view.toSql() has some syntax error. - try { - newView.init(); - } catch (UserException e) { - throw new DdlException("failed to init view stmt, reason=" + e.getMessage()); - } - if (!((Database) db).createTableWithLock(newView, false, stmt.isSetIfNotExists()).first) { throw new DdlException("Failed to create view[" + tableName + "]."); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterViewInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterViewInfo.java index eea1afc2844a8d5..eb8af115ac0c14b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterViewInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterViewInfo.java @@ -78,18 +78,16 @@ public void init(ConnectContext ctx) throws UserException { /**translateToLegacyStmt*/ public AlterViewStmt translateToLegacyStmt(ConnectContext ctx) { - List cols = Lists.newArrayList(); - for (SimpleColumnDefinition def : simpleColumnDefinitions) { - cols.add(def.translateToColWithComment()); - } - AlterViewStmt alterViewStmt = new AlterViewStmt(viewName.transferToTableName(), cols, - null); // expand star(*) in project list and replace table name with qualifier String rewrittenSql = rewriteSql(ctx.getStatementContext().getIndexInSqlToString(), querySql); - // rewrite project alias rewrittenSql = rewriteProjectsToUserDefineAlias(rewrittenSql); - + checkViewSql(rewrittenSql); + List cols = Lists.newArrayList(); + for (SimpleColumnDefinition def : simpleColumnDefinitions) { + cols.add(def.translateToColWithComment()); + } + AlterViewStmt alterViewStmt = new AlterViewStmt(viewName.transferToTableName(), cols, null); alterViewStmt.setInlineViewDef(rewrittenSql); alterViewStmt.setFinalColumns(finalCols); return alterViewStmt; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BaseViewInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BaseViewInfo.java index 9c4b8670bb6e868..8df0f83b0b641c6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BaseViewInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/BaseViewInfo.java @@ -389,4 +389,8 @@ public Void visitSlot(Slot slot, StatementContext ctx) { return null; } } + + protected void checkViewSql(String viewSql) { + new NereidsParser().parseSingle(viewSql); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java index c0260ea682fe8ee..d3cd4293f4160eb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java @@ -73,18 +73,17 @@ public void init(ConnectContext ctx) throws UserException { /**translateToLegacyStmt*/ public CreateViewStmt translateToLegacyStmt(ConnectContext ctx) { + // expand star(*) in project list and replace table name with qualifier + String rewrittenSql = rewriteSql(ctx.getStatementContext().getIndexInSqlToString(), querySql); + // rewrite project alias + rewrittenSql = rewriteProjectsToUserDefineAlias(rewrittenSql); + checkViewSql(rewrittenSql); List cols = Lists.newArrayList(); for (SimpleColumnDefinition def : simpleColumnDefinitions) { cols.add(def.translateToColWithComment()); } CreateViewStmt createViewStmt = new CreateViewStmt(ifNotExists, orReplace, viewName.transferToTableName(), cols, comment, null); - // expand star(*) in project list and replace table name with qualifier - String rewrittenSql = rewriteSql(ctx.getStatementContext().getIndexInSqlToString(), querySql); - - // rewrite project alias - rewrittenSql = rewriteProjectsToUserDefineAlias(rewrittenSql); - createViewStmt.setInlineViewDef(rewrittenSql); createViewStmt.setFinalColumns(finalCols); return createViewStmt; diff --git a/regression-test/data/ddl_p0/create_view_nereids/create_view_use_mv.out b/regression-test/data/ddl_p0/create_view_nereids/create_view_use_mv.out new file mode 100644 index 000000000000000..c4df7484022baae --- /dev/null +++ b/regression-test/data/ddl_p0/create_view_nereids/create_view_use_mv.out @@ -0,0 +1,15 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !create_view_from_mv -- +\N 99.50 99.50 99.50 1 \N \N \N \N 1 \N \N +1 208.70 109.20 99.50 3 1 3 1 1 3 \N \N +2 218.40 109.20 109.20 2 2 2 2 2 1 \N \N +3 298.50 99.50 99.50 3 3 6 3 3 2 \N \N +4 99.50 99.50 99.50 1 \N 4 4 4 1 \N \N + +-- !alter_view_from_mv -- +\N 99.50 99.50 99.50 1 \N \N \N \N 1 \N \N +1 208.70 109.20 99.50 3 1 3 1 1 3 \N \N +2 218.40 109.20 109.20 2 2 2 2 2 1 \N \N +3 298.50 99.50 99.50 3 3 6 3 3 2 \N \N +4 99.50 99.50 99.50 1 \N 4 4 4 1 \N \N + diff --git a/regression-test/data/ddl_p0/create_view_nereids/test_create_view_nereids.out b/regression-test/data/ddl_p0/create_view_nereids/test_create_view_nereids.out new file mode 100644 index 000000000000000..5bdfe66d5aa672e --- /dev/null +++ b/regression-test/data/ddl_p0/create_view_nereids/test_create_view_nereids.out @@ -0,0 +1,397 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !test_view_1 -- +1 [1, 2, 3] +2 [10, -2, 8] +3 [-1, 20, 0] + +-- !test_view_2 -- +1 [1, 2, 3] [1, 1, 1] +2 [10, -2, 8] [1, 0, 1] +3 [-1, 20, 0] [0, 1, 0] + +-- !test_view_3 -- +1 [1, 2, 3] [1, 2, 3] [1, 2, 3] +2 [10, -2, 8] [10, 8] [10, 8] +3 [-1, 20, 0] [20] [20] + +-- !test_view_4 -- +1 [1, 2, 3] [1, 2, 3] [1, 2, 3] +2 [10, -2, 8] [10, 8] [10, 8] +3 [-1, 20, 0] [20] [20] + +-- !test_view_5 -- +1 [1, 2, 3] [1, 1, 1] +2 [10, -2, 8] [1, 0, 1] +3 [-1, 20, 0] [0, 1, 0] + +-- !test_view_6 -- +v1 CREATE VIEW `v1` AS SELECT\n `internal`.`regression_test_ddl_p0_create_view_nereids`.`view_column_name_test_nereids`.`error_code`, \n 1, \n 'string', \n now(), \n dayofyear(`internal`.`regression_test_ddl_p0_create_view_nereids`.`view_column_name_test_nereids`.`op_time`), \n cast (`internal`.`regression_test_ddl_p0_create_view_nereids`.`view_column_name_test_nereids`.`source` AS BIGINT), \n min(`internal`.`regression_test_ddl_p0_create_view_nereids`.`view_column_name_test_nereids`.`timestamp`) OVER (\n ORDER BY \n `internal`.`regression_test_ddl_p0_create_view_nereids`.`view_column_name_test_nereids`.`op_time` DESC ROWS BETWEEN UNBOUNDED PRECEDING\n AND 1 FOLLOWING\n ), \n 1 > 2,\n 2 + 3,\n 1 IN (1, 2, 3, 4), \n `internal`.`regression_test_ddl_p0_create_view_nereids`.`view_column_name_test_nereids`.`remark` LIKE '%like', \n CASE WHEN `internal`.`regression_test_ddl_p0_create_view_nereids`.`view_column_name_test_nereids`.`remark` = 's' THEN 1 ELSE 2 END,\n TRUE | FALSE \n FROM \n `internal`.`regression_test_ddl_p0_create_view_nereids`.`view_column_name_test_nereids`; + +-- !test_with_as -- +1 1 2 +1 1 4 +1 3 6 +2 1 3 +2 1 4 +2 1 7 +2 3 5 +2 3 9 +2 4 2 +3 2 8 +3 5 \N +3 5 6 +3 5 6 +3 5 8 +4 5 6 +6 \N 6 +6 7 1 + +-- !test_with_as_sql -- +test_view_with_as CREATE VIEW `test_view_with_as` AS (\n with `t1` as (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`), `t2` as (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`), \n `t3` as (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`) SELECT `t1`.`pk`, `t1`.`a`, `t1`.`b` FROM t1); utf8mb4 utf8mb4_0900_bin + +-- !test_union -- +1 1 2 +1 1 2 +1 1 4 +1 1 4 +1 3 6 +1 3 6 +2 1 3 +2 1 3 +2 1 4 +2 1 4 +2 1 7 +2 1 7 +2 3 5 +2 3 5 +2 3 9 +2 3 9 +2 4 2 +2 4 2 +3 2 8 +3 2 8 +3 5 \N +3 5 \N +3 5 6 +3 5 6 +3 5 6 +3 5 6 +3 5 8 +3 5 8 +4 5 6 +4 5 6 +6 \N 6 +6 \N 6 +6 7 1 +6 7 1 + +-- !test_union_sql -- +test_view_union CREATE VIEW `test_view_union` AS (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk` AS `c1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a` AS `c2`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` AS `c3` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view` Union all SELECT `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` FROM `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`); utf8mb4 utf8mb4_0900_bin + +-- !test_count_star -- +17 + +-- !test_count_star_sql -- +test_view_count_star CREATE VIEW `test_view_count_star` AS (select count(*) AS `c1` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view` having count(*) > 0); utf8mb4 utf8mb4_0900_bin + +-- !test_expression -- +\N \N 6 +2 4 2 +2 4 3 +2 4 4 +2 4 4 +2 4 7 +3 5 8 +4 6 5 +4 6 6 +4 6 9 +5 7 2 +6 8 \N +6 8 6 +6 8 6 +6 8 6 +6 8 8 +8 10 1 + +-- !test_expression_sql -- +test_view_expression CREATE VIEW `test_view_expression` AS (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`+1 AS `c1`, abs(`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`)+2+1 AS `c2`, cast(`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` as varchar(10)) AS `c3` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`); utf8mb4 utf8mb4_0900_bin + +-- !test_alias -- +\N \N 6 +2 4 2 +2 4 3 +2 4 4 +2 4 4 +2 4 7 +3 5 8 +4 6 5 +4 6 6 +4 6 9 +5 7 2 +6 8 \N +6 8 6 +6 8 6 +6 8 6 +6 8 8 +8 10 1 + +-- !test_alias_sql -- +test_view_alias CREATE VIEW `test_view_alias` AS (\n select `t`.`c8` AS `c1`, `t`.`c2` AS `c2`, `t`.`c1` AS `c3` from (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`+1 `c8`,abs(`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`)+2+1 as `c2`, cast(`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` as varchar(10)) as `c1` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`) `t`); utf8mb4 utf8mb4_0900_bin + +-- !test_star_except -- +\N 6 +1 2 +1 3 +1 4 +1 4 +1 7 +2 8 +3 5 +3 6 +3 9 +4 2 +5 \N +5 6 +5 6 +5 6 +5 8 +7 1 + +-- !test_star_except_sql -- +test_view_star_except CREATE VIEW `test_view_star_except` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`; utf8mb4 utf8mb4_0900_bin + +-- !test_create_view_from_view -- +1 1 2 +1 1 4 +1 3 6 +2 1 3 +2 1 4 +2 1 7 +2 3 5 +2 3 9 +2 4 2 +3 2 8 +3 5 \N +3 5 6 +3 5 6 +3 5 8 +4 5 6 +6 \N 6 +6 7 1 + +-- !test_create_view_from_view_sql -- +test_view_from_view CREATE VIEW `test_view_from_view` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`test_view_with_as`.`pk` AS `c1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`test_view_with_as`.`a` AS `c2`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`test_view_with_as`.`b` AS `c3` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`test_view_with_as`; utf8mb4 utf8mb4_0900_bin + +-- !test_backquote_in_view_define -- +\N 6 +1 2 +1 3 +1 4 +1 4 +1 7 +2 8 +3 5 +3 6 +3 9 +4 2 +5 \N +5 6 +5 6 +5 6 +5 8 +7 1 + +-- !test_backquote_in_view_define_sql -- +test_backquote_in_view_define CREATE VIEW `test_backquote_in_view_define` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a` AS `abc`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` AS `c2` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`; utf8mb4 utf8mb4_0900_bin + +-- !test_backquote_in_table_alias -- +\N 6 +1 2 +1 3 +1 4 +1 4 +1 7 +2 8 +3 5 +3 6 +3 9 +4 2 +5 \N +5 6 +5 6 +5 6 +5 8 +7 1 + +-- !test_backquote_in_table_alias_sql -- +test_backquote_in_table_alias CREATE VIEW `test_backquote_in_table_alias` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`ab``c`.`a` AS `c1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`ab``c`.`b` AS `c2` from (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`,`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`) `ab``c`; utf8mb4 utf8mb4_0900_bin + +-- !test_invalid_column_name_in_table -- +\N 6 +1 2 +1 3 +1 4 +1 4 +1 7 +2 8 +3 5 +3 6 +3 9 +4 2 +5 \N +5 6 +5 6 +5 6 +5 8 +7 1 + +-- !test_invalid_column_name_in_table_define_sql -- +test_invalid_column_name_in_table CREATE VIEW `test_invalid_column_name_in_table` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a` ,`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`; utf8mb4 utf8mb4_0900_bin + +-- !test_generate -- +1 10 A 30 +1 10 A 60 +2 20 B 30 +2 20 B 60 +3 30 C 30 +3 30 C 60 +4 40 D 30 +4 40 D 60 + +-- !test_generate_sql -- +test_view_generate CREATE VIEW `test_view_generate` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`value1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`value2`, `t1`.`age` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1` lateral view EXPLODE(ARRAY(30,60)) `t1` as `age`; utf8mb4 utf8mb4_0900_bin + +-- !test_generate_with_column -- +1 10 A 0 +2 20 B 0 +2 20 B 1 +3 30 C 0 +3 30 C 1 +3 30 C 2 +4 40 D 0 +4 40 D 1 +4 40 D 2 +4 40 D 3 + +-- !test_generate_with_column_sql -- +test_view_generate_with_column CREATE VIEW `test_view_generate_with_column` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`value1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`value2`, `t1`.`age` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1` lateral view EXPLODE_numbers(`internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`) `t1` as `age`; utf8mb4 utf8mb4_0900_bin + +-- !test_col_alias -- +1 10 +2 20 +3 30 +4 40 + +-- !test_col_alias_sql -- +test_view_col_alias CREATE VIEW `test_view_col_alias` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id` as `c1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`value1` as `c2` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`; utf8mb4 utf8mb4_0900_bin + +-- !test_col_alias_with_specific_name -- +1 10 +2 20 +3 30 +4 40 + +-- !test_col_alias_with_specific_name_sql -- +test_view_col_alias_specific_name CREATE VIEW `test_view_col_alias_specific_name` AS select `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id` AS `col1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`value1` AS `col2` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`; utf8mb4 utf8mb4_0900_bin + +-- !test_table_alias -- +1 10 +2 20 +3 30 +4 40 + +-- !test_table_alias_sql -- +test_view_table_alias CREATE VIEW `test_view_table_alias` AS select `t`.`c1`, `t`.`c2` from (\n select `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id` as `c1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`value1` as `c2` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1` limit 10) as `t`; utf8mb4 utf8mb4_0900_bin + +-- !test_join_table_alias -- +1 10 +2 20 +3 30 +4 40 + +-- !test_join_table_alias_sql -- +test_view_join_table_alias CREATE VIEW `test_view_join_table_alias` AS select `t`.`c1`, `t`.`c2` from (\n select `internal`.`regression_test_ddl_p0_create_view_nereids`.`t1`.`id` as `c1`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`t1`.`value1` as `c2` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1` `t1` inner join `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table2` `t2` on `internal`.`regression_test_ddl_p0_create_view_nereids`.`t1`.`id`=`internal`.`regression_test_ddl_p0_create_view_nereids`.`t2`.`id` limit 10) as `t`; utf8mb4 utf8mb4_0900_bin + +-- !test_alias_udf -- +1****1 1 +2****2 2 +3****3 3 +4****4 4 + +-- !test_alias_udf_sql -- +test_view_alias_udf CREATE VIEW `test_view_alias_udf` AS (select `regression_test_ddl_p0_create_view_nereids`.`alias_function_create_view_test`(`internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`) as `c1`,abs(`internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`) from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`); utf8mb4 utf8mb4_0900_bin + +-- !test_alias_with_db_udf -- +1****1 1 +2****2 2 +3****3 3 +4****4 4 + +-- !test_alias_with_db_udf_sql -- +test_view_alias_udf_with_db CREATE VIEW `test_view_alias_udf_with_db` AS (select `regression_test_ddl_p0_create_view_nereids`.`alias_function_create_view_test`(`internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`) as `c1`,abs(`internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`) from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`); utf8mb4 utf8mb4_0900_bin + +-- !test_udf_sql -- +test_view_udf CREATE VIEW `test_view_udf` AS (select `regression_test_ddl_p0_create_view_nereids`.`alias_function_create_view_test`(`internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`) as `c1`, `regression_test_ddl_p0_create_view_nereids`.`java_udf_create_view_test`('2011-01-01','2011-01-02'),\n `regression_test_ddl_p0_create_view_nereids`.`java_udf_create_view_test`('2011-01-01','2011-01-03') from `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`); utf8mb4 utf8mb4_0900_bin + +-- !test_udf -- +1****1 false false +2****2 false false +3****3 false false +4****4 false false + +-- !test_with_as_with_columns -- +1 1 2 +1 1 4 +1 3 6 +2 1 3 +2 1 4 +2 1 7 +2 3 5 +2 3 9 +2 4 2 +3 2 8 +3 5 \N +3 5 6 +3 5 6 +3 5 8 +4 5 6 +6 \N 6 +6 7 1 + +-- !test_with_as_with_columns_sql -- +test_view_with_as_with_columns CREATE VIEW `test_view_with_as_with_columns` AS (\n with `t1`(`c1`,`c2`,`c3`) as (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`), `t2` as (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`), \n `t3` as (select `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`) SELECT `t1`.`c1`, `t1`.`c2`, `t1`.`c3` FROM t1); utf8mb4 utf8mb4_0900_bin + +-- !test_having -- +2 2 +3 3 +4 2 +4 4 +5 5 +6 3 +7 7 +8 4 +9 3 +10 5 +14 7 +15 5 + +-- !test_having_sql -- +test_having CREATE VIEW `test_having` AS select sum(`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`) over(partition by `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a` order by `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`) as `c1` , `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a` from `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view` group by grouping sets((`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`),(`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`b`),(`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`pk`,`internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`)) having `internal`.`regression_test_ddl_p0_create_view_nereids`.`mal_test_view`.`a`>1; utf8mb4 utf8mb4_0900_bin + +-- !complicated_view1 -- +1 100 1 +2 200 1 + +-- !complicated_view1_sql -- +test_view_complicated CREATE VIEW `test_view_complicated` AS SELECT `internal`.`regression_test_ddl_p0_create_view_nereids`.`t`.`id`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`t`.`value3`, `t`.`row_num` FROM (\n SELECT `internal`.`regression_test_ddl_p0_create_view_nereids`.`t1`.`id`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`tt`.`value3`, ROW_NUMBER() OVER (PARTITION BY `internal`.`regression_test_ddl_p0_create_view_nereids`.`t1`.`id` ORDER BY `internal`.`regression_test_ddl_p0_create_view_nereids`.`tt`.`value3` DESC) as `row_num`\n FROM (SELECT `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id` FROM `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1` GROUP BY `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table1`.`id`) `t1`\n FULL OUTER JOIN (SELECT `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table2`.`value3`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table2`.`id`, MAX(`internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table2`.`value4`) FROM `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table2` GROUP BY `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table2`.`value3`, `internal`.`regression_test_ddl_p0_create_view_nereids`.`create_view_table2`.`id`) `tt`\n ON `internal`.`regression_test_ddl_p0_create_view_nereids`.`tt`.`id` = `internal`.`regression_test_ddl_p0_create_view_nereids`.`t1`.`id`\n ORDER BY `internal`.`regression_test_ddl_p0_create_view_nereids`.`t1`.`id`\n ) `t`\n WHERE `internal`.`regression_test_ddl_p0_create_view_nereids`.`t`.`value3` < 280 AND (`internal`.`regression_test_ddl_p0_create_view_nereids`.`t`.`id` < 3 or `internal`.`regression_test_ddl_p0_create_view_nereids`.`t`.`id` >8); utf8mb4 utf8mb4_0900_bin + +-- !nullable -- +1 小区A 10 1 2024-09-01T09:00 2024-09-01T10:00 +2 小区B 11 1 2024-09-01T09:00 2024-09-01T10:00 +3 小区C \N 1 2024-09-01T09:00 2024-09-01T10:00 + +-- !nullable_view_with_cols -- +1 小区A 10 1 2024-09-01T09:00 2024-09-01T10:00 +2 小区B 11 1 2024-09-01T09:00 2024-09-01T10:00 +3 小区C \N 1 2024-09-01T09:00 2024-09-01T10:00 + diff --git a/regression-test/data/ddl_p0/test_create_view_nereids.out b/regression-test/data/ddl_p0/test_create_view_nereids.out deleted file mode 100644 index 1b2ba19d93761d6..000000000000000 --- a/regression-test/data/ddl_p0/test_create_view_nereids.out +++ /dev/null @@ -1,397 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !test_view_1 -- -1 [1, 2, 3] -2 [10, -2, 8] -3 [-1, 20, 0] - --- !test_view_2 -- -1 [1, 2, 3] [1, 1, 1] -2 [10, -2, 8] [1, 0, 1] -3 [-1, 20, 0] [0, 1, 0] - --- !test_view_3 -- -1 [1, 2, 3] [1, 2, 3] [1, 2, 3] -2 [10, -2, 8] [10, 8] [10, 8] -3 [-1, 20, 0] [20] [20] - --- !test_view_4 -- -1 [1, 2, 3] [1, 2, 3] [1, 2, 3] -2 [10, -2, 8] [10, 8] [10, 8] -3 [-1, 20, 0] [20] [20] - --- !test_view_5 -- -1 [1, 2, 3] [1, 1, 1] -2 [10, -2, 8] [1, 0, 1] -3 [-1, 20, 0] [0, 1, 0] - --- !test_view_6 -- -v1 CREATE VIEW `v1` AS SELECT\n `internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`.`error_code`, \n 1, \n 'string', \n now(), \n dayofyear(`internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`.`op_time`), \n cast (`internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`.`source` AS BIGINT), \n min(`internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`.`timestamp`) OVER (\n ORDER BY \n `internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`.`op_time` DESC ROWS BETWEEN UNBOUNDED PRECEDING\n AND 1 FOLLOWING\n ), \n 1 > 2,\n 2 + 3,\n 1 IN (1, 2, 3, 4), \n `internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`.`remark` LIKE '%like', \n CASE WHEN `internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`.`remark` = 's' THEN 1 ELSE 2 END,\n TRUE | FALSE \n FROM \n `internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`; - --- !test_with_as -- -1 1 2 -1 1 4 -1 3 6 -2 1 3 -2 1 4 -2 1 7 -2 3 5 -2 3 9 -2 4 2 -3 2 8 -3 5 \N -3 5 6 -3 5 6 -3 5 8 -4 5 6 -6 \N 6 -6 7 1 - --- !test_with_as_sql -- -test_view_with_as CREATE VIEW `test_view_with_as` AS (\n with `t1` as (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`), `t2` as (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`), \n `t3` as (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`) SELECT `t1`.`pk`, `t1`.`a`, `t1`.`b` FROM t1); utf8mb4 utf8mb4_0900_bin - --- !test_union -- -1 1 2 -1 1 2 -1 1 4 -1 1 4 -1 3 6 -1 3 6 -2 1 3 -2 1 3 -2 1 4 -2 1 4 -2 1 7 -2 1 7 -2 3 5 -2 3 5 -2 3 9 -2 3 9 -2 4 2 -2 4 2 -3 2 8 -3 2 8 -3 5 \N -3 5 \N -3 5 6 -3 5 6 -3 5 6 -3 5 6 -3 5 8 -3 5 8 -4 5 6 -4 5 6 -6 \N 6 -6 \N 6 -6 7 1 -6 7 1 - --- !test_union_sql -- -test_view_union CREATE VIEW `test_view_union` AS (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk` AS `c1`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a` AS `c2`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` AS `c3` from `internal`.`regression_test_ddl_p0`.`mal_test_view` Union all SELECT `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` FROM `internal`.`regression_test_ddl_p0`.`mal_test_view`); utf8mb4 utf8mb4_0900_bin - --- !test_count_star -- -17 - --- !test_count_star_sql -- -test_view_count_star CREATE VIEW `test_view_count_star` AS (select count(*) AS `c1` from `internal`.`regression_test_ddl_p0`.`mal_test_view` having count(*) > 0); utf8mb4 utf8mb4_0900_bin - --- !test_expression -- -\N \N 6 -2 4 2 -2 4 3 -2 4 4 -2 4 4 -2 4 7 -3 5 8 -4 6 5 -4 6 6 -4 6 9 -5 7 2 -6 8 \N -6 8 6 -6 8 6 -6 8 6 -6 8 8 -8 10 1 - --- !test_expression_sql -- -test_view_expression CREATE VIEW `test_view_expression` AS (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`+1 AS `c1`, abs(`internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`)+2+1 AS `c2`, cast(`internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` as varchar(10)) AS `c3` from `internal`.`regression_test_ddl_p0`.`mal_test_view`); utf8mb4 utf8mb4_0900_bin - --- !test_alias -- -\N \N 6 -2 4 2 -2 4 3 -2 4 4 -2 4 4 -2 4 7 -3 5 8 -4 6 5 -4 6 6 -4 6 9 -5 7 2 -6 8 \N -6 8 6 -6 8 6 -6 8 6 -6 8 8 -8 10 1 - --- !test_alias_sql -- -test_view_alias CREATE VIEW `test_view_alias` AS (\n select `t`.`c8` AS `c1`, `t`.`c2` AS `c2`, `t`.`c1` AS `c3` from (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`+1 `c8`,abs(`internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`)+2+1 as `c2`, cast(`internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` as varchar(10)) as `c1` from `internal`.`regression_test_ddl_p0`.`mal_test_view`) `t`); utf8mb4 utf8mb4_0900_bin - --- !test_star_except -- -\N 6 -1 2 -1 3 -1 4 -1 4 -1 7 -2 8 -3 5 -3 6 -3 9 -4 2 -5 \N -5 6 -5 6 -5 6 -5 8 -7 1 - --- !test_star_except_sql -- -test_view_star_except CREATE VIEW `test_view_star_except` AS select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`; utf8mb4 utf8mb4_0900_bin - --- !test_create_view_from_view -- -1 1 2 -1 1 4 -1 3 6 -2 1 3 -2 1 4 -2 1 7 -2 3 5 -2 3 9 -2 4 2 -3 2 8 -3 5 \N -3 5 6 -3 5 6 -3 5 8 -4 5 6 -6 \N 6 -6 7 1 - --- !test_create_view_from_view_sql -- -test_view_from_view CREATE VIEW `test_view_from_view` AS select `internal`.`regression_test_ddl_p0`.`test_view_with_as`.`pk` AS `c1`, `internal`.`regression_test_ddl_p0`.`test_view_with_as`.`a` AS `c2`, `internal`.`regression_test_ddl_p0`.`test_view_with_as`.`b` AS `c3` from `internal`.`regression_test_ddl_p0`.`test_view_with_as`; utf8mb4 utf8mb4_0900_bin - --- !test_backquote_in_view_define -- -\N 6 -1 2 -1 3 -1 4 -1 4 -1 7 -2 8 -3 5 -3 6 -3 9 -4 2 -5 \N -5 6 -5 6 -5 6 -5 8 -7 1 - --- !test_backquote_in_view_define_sql -- -test_backquote_in_view_define CREATE VIEW `test_backquote_in_view_define` AS select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a` AS `abc`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` AS `c2` from `internal`.`regression_test_ddl_p0`.`mal_test_view`; utf8mb4 utf8mb4_0900_bin - --- !test_backquote_in_table_alias -- -\N 6 -1 2 -1 3 -1 4 -1 4 -1 7 -2 8 -3 5 -3 6 -3 9 -4 2 -5 \N -5 6 -5 6 -5 6 -5 8 -7 1 - --- !test_backquote_in_table_alias_sql -- -test_backquote_in_table_alias CREATE VIEW `test_backquote_in_table_alias` AS select `internal`.`regression_test_ddl_p0`.`ab``c`.`a` AS `c1`, `internal`.`regression_test_ddl_p0`.`ab``c`.`b` AS `c2` from (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`,`internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`) `ab``c`; utf8mb4 utf8mb4_0900_bin - --- !test_invalid_column_name_in_table -- -\N 6 -1 2 -1 3 -1 4 -1 4 -1 7 -2 8 -3 5 -3 6 -3 9 -4 2 -5 \N -5 6 -5 6 -5 6 -5 8 -7 1 - --- !test_invalid_column_name_in_table_define_sql -- -test_invalid_column_name_in_table CREATE VIEW `test_invalid_column_name_in_table` AS select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a` ,`internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`; utf8mb4 utf8mb4_0900_bin - --- !test_generate -- -1 10 A 30 -1 10 A 60 -2 20 B 30 -2 20 B 60 -3 30 C 30 -3 30 C 60 -4 40 D 30 -4 40 D 60 - --- !test_generate_sql -- -test_view_generate CREATE VIEW `test_view_generate` AS select `internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`, `internal`.`regression_test_ddl_p0`.`create_view_table1`.`value1`, `internal`.`regression_test_ddl_p0`.`create_view_table1`.`value2`, `t1`.`age` from `internal`.`regression_test_ddl_p0`.`create_view_table1` lateral view EXPLODE(ARRAY(30,60)) `t1` as `age`; utf8mb4 utf8mb4_0900_bin - --- !test_generate_with_column -- -1 10 A 0 -2 20 B 0 -2 20 B 1 -3 30 C 0 -3 30 C 1 -3 30 C 2 -4 40 D 0 -4 40 D 1 -4 40 D 2 -4 40 D 3 - --- !test_generate_with_column_sql -- -test_view_generate_with_column CREATE VIEW `test_view_generate_with_column` AS select `internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`, `internal`.`regression_test_ddl_p0`.`create_view_table1`.`value1`, `internal`.`regression_test_ddl_p0`.`create_view_table1`.`value2`, `t1`.`age` from `internal`.`regression_test_ddl_p0`.`create_view_table1` lateral view EXPLODE_numbers(`internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`) `t1` as `age`; utf8mb4 utf8mb4_0900_bin - --- !test_col_alias -- -1 10 -2 20 -3 30 -4 40 - --- !test_col_alias_sql -- -test_view_col_alias CREATE VIEW `test_view_col_alias` AS select `internal`.`regression_test_ddl_p0`.`create_view_table1`.`id` as `c1`, `internal`.`regression_test_ddl_p0`.`create_view_table1`.`value1` as `c2` from `internal`.`regression_test_ddl_p0`.`create_view_table1`; utf8mb4 utf8mb4_0900_bin - --- !test_col_alias_with_specific_name -- -1 10 -2 20 -3 30 -4 40 - --- !test_col_alias_with_specific_name_sql -- -test_view_col_alias_specific_name CREATE VIEW `test_view_col_alias_specific_name` AS select `internal`.`regression_test_ddl_p0`.`create_view_table1`.`id` AS `col1`, `internal`.`regression_test_ddl_p0`.`create_view_table1`.`value1` AS `col2` from `internal`.`regression_test_ddl_p0`.`create_view_table1`; utf8mb4 utf8mb4_0900_bin - --- !test_table_alias -- -1 10 -2 20 -3 30 -4 40 - --- !test_table_alias_sql -- -test_view_table_alias CREATE VIEW `test_view_table_alias` AS select `t`.`c1`, `t`.`c2` from (\n select `internal`.`regression_test_ddl_p0`.`create_view_table1`.`id` as `c1`, `internal`.`regression_test_ddl_p0`.`create_view_table1`.`value1` as `c2` from `internal`.`regression_test_ddl_p0`.`create_view_table1` limit 10) as `t`; utf8mb4 utf8mb4_0900_bin - --- !test_join_table_alias -- -1 10 -2 20 -3 30 -4 40 - --- !test_join_table_alias_sql -- -test_view_join_table_alias CREATE VIEW `test_view_join_table_alias` AS select `t`.`c1`, `t`.`c2` from (\n select `internal`.`regression_test_ddl_p0`.`t1`.`id` as `c1`, `internal`.`regression_test_ddl_p0`.`t1`.`value1` as `c2` from `internal`.`regression_test_ddl_p0`.`create_view_table1` `t1` inner join `internal`.`regression_test_ddl_p0`.`create_view_table2` `t2` on `internal`.`regression_test_ddl_p0`.`t1`.`id`=`internal`.`regression_test_ddl_p0`.`t2`.`id` limit 10) as `t`; utf8mb4 utf8mb4_0900_bin - --- !test_alias_udf -- -1****1 1 -2****2 2 -3****3 3 -4****4 4 - --- !test_alias_udf_sql -- -test_view_alias_udf CREATE VIEW `test_view_alias_udf` AS (select `regression_test_ddl_p0`.`alias_function_create_view_test`(`internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`) as `c1`,abs(`internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`) from `internal`.`regression_test_ddl_p0`.`create_view_table1`); utf8mb4 utf8mb4_0900_bin - --- !test_alias_with_db_udf -- -1****1 1 -2****2 2 -3****3 3 -4****4 4 - --- !test_alias_with_db_udf_sql -- -test_view_alias_udf_with_db CREATE VIEW `test_view_alias_udf_with_db` AS (select `regression_test_ddl_p0`.`alias_function_create_view_test`(`internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`) as `c1`,abs(`internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`) from `internal`.`regression_test_ddl_p0`.`create_view_table1`); utf8mb4 utf8mb4_0900_bin - --- !test_udf_sql -- -test_view_udf CREATE VIEW `test_view_udf` AS (select `regression_test_ddl_p0`.`alias_function_create_view_test`(`internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`) as `c1`, `regression_test_ddl_p0`.`java_udf_create_view_test`('2011-01-01','2011-01-02'),\n `regression_test_ddl_p0`.`java_udf_create_view_test`('2011-01-01','2011-01-03') from `internal`.`regression_test_ddl_p0`.`create_view_table1`); utf8mb4 utf8mb4_0900_bin - --- !test_udf -- -1****1 false false -2****2 false false -3****3 false false -4****4 false false - --- !test_with_as_with_columns -- -1 1 2 -1 1 4 -1 3 6 -2 1 3 -2 1 4 -2 1 7 -2 3 5 -2 3 9 -2 4 2 -3 2 8 -3 5 \N -3 5 6 -3 5 6 -3 5 8 -4 5 6 -6 \N 6 -6 7 1 - --- !test_with_as_with_columns_sql -- -test_view_with_as_with_columns CREATE VIEW `test_view_with_as_with_columns` AS (\n with `t1`(`c1`,`c2`,`c3`) as (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`), `t2` as (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`), \n `t3` as (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`) SELECT `t1`.`c1`, `t1`.`c2`, `t1`.`c3` FROM t1); utf8mb4 utf8mb4_0900_bin - --- !test_having -- -2 2 -3 3 -4 2 -4 4 -5 5 -6 3 -7 7 -8 4 -9 3 -10 5 -14 7 -15 5 - --- !test_having_sql -- -test_having CREATE VIEW `test_having` AS select sum(`internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`) over(partition by `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a` order by `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`) as `c1` , `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a` from `internal`.`regression_test_ddl_p0`.`mal_test_view` group by grouping sets((`internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`),(`internal`.`regression_test_ddl_p0`.`mal_test_view`.`b`),(`internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`,`internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`)) having `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`>1; utf8mb4 utf8mb4_0900_bin - --- !complicated_view1 -- -1 100 1 -2 200 1 - --- !complicated_view1_sql -- -test_view_complicated CREATE VIEW `test_view_complicated` AS SELECT `internal`.`regression_test_ddl_p0`.`t`.`id`, `internal`.`regression_test_ddl_p0`.`t`.`value3`, `t`.`row_num` FROM (\n SELECT `internal`.`regression_test_ddl_p0`.`t1`.`id`, `internal`.`regression_test_ddl_p0`.`tt`.`value3`, ROW_NUMBER() OVER (PARTITION BY `internal`.`regression_test_ddl_p0`.`t1`.`id` ORDER BY `internal`.`regression_test_ddl_p0`.`tt`.`value3` DESC) as `row_num`\n FROM (SELECT `internal`.`regression_test_ddl_p0`.`create_view_table1`.`id` FROM `internal`.`regression_test_ddl_p0`.`create_view_table1` GROUP BY `internal`.`regression_test_ddl_p0`.`create_view_table1`.`id`) `t1`\n FULL OUTER JOIN (SELECT `internal`.`regression_test_ddl_p0`.`create_view_table2`.`value3`, `internal`.`regression_test_ddl_p0`.`create_view_table2`.`id`, MAX(`internal`.`regression_test_ddl_p0`.`create_view_table2`.`value4`) FROM `internal`.`regression_test_ddl_p0`.`create_view_table2` GROUP BY `internal`.`regression_test_ddl_p0`.`create_view_table2`.`value3`, `internal`.`regression_test_ddl_p0`.`create_view_table2`.`id`) `tt`\n ON `internal`.`regression_test_ddl_p0`.`tt`.`id` = `internal`.`regression_test_ddl_p0`.`t1`.`id`\n ORDER BY `internal`.`regression_test_ddl_p0`.`t1`.`id`\n ) `t`\n WHERE `internal`.`regression_test_ddl_p0`.`t`.`value3` < 280 AND (`internal`.`regression_test_ddl_p0`.`t`.`id` < 3 or `internal`.`regression_test_ddl_p0`.`t`.`id` >8); utf8mb4 utf8mb4_0900_bin - --- !nullable -- -1 小区A 10 1 2024-09-01T09:00 2024-09-01T10:00 -2 小区B 11 1 2024-09-01T09:00 2024-09-01T10:00 -3 小区C \N 1 2024-09-01T09:00 2024-09-01T10:00 - --- !nullable_view_with_cols -- -1 小区A 10 1 2024-09-01T09:00 2024-09-01T10:00 -2 小区B 11 1 2024-09-01T09:00 2024-09-01T10:00 -3 小区C \N 1 2024-09-01T09:00 2024-09-01T10:00 - diff --git a/regression-test/suites/ddl_p0/create_view_nereids/create_view_use_mv.groovy b/regression-test/suites/ddl_p0/create_view_nereids/create_view_use_mv.groovy new file mode 100644 index 000000000000000..295b195aa589547 --- /dev/null +++ b/regression-test/suites/ddl_p0/create_view_nereids/create_view_use_mv.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("create_view_use_mv") { + sql "drop table if exists orders" + sql """create table orders ( o_orderkey bigint null, o_custkey int null, o_orderstatus varchar(1) null, + o_totalprice decimal(15,2) null, o_orderpriority varchar(15) null, o_clerk varchar(15) null, o_shippriority int null, + o_comment varchar(79) null, o_orderdate date not null) engine=olap duplicate key(o_orderkey,o_custkey) + comment 'olap' distributed by hash(o_orderkey) buckets 96 properties("replication_num"="1")""" + + sql "drop table if exists lineitem" + sql """create table lineitem ( + l_orderkey bigint null, l_linenumber int null, l_partkey int null, l_suppkey int null, l_quantity decimal(15,2) null,l_extendedprice decimal(15,2) null, + l_discount decimal(15,2) null, l_tax decimal(15,2) null, l_returnflag varchar(1) null, l_linestatus varchar(1) null, l_commitdate date null, l_receiptdate date null, + l_shipnstruct varchar(25) null, l_shipmode varchar(10) null, l_comment varchar(44) null,l_shipdate date not null) engine=olap + duplicate key(l_orderkey, l_linenumber,l_partkey, l_suppkey) + distributed by hash(l_orderkey) buckets 96 + properties("replication_num"="1");""" + + sql """insert into orders values (null, 1, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'),(1, null, 'o', 109.2, 'c','d',2, 'mm', '2023-10-17'), + (3, 3, null, 99.5, 'a', 'b', 1, 'yy', '2023-10-19'), + (1, 2, 'o', null, 'a', 'b', 1, 'yy', '2023-10-20'), + (2, 3, 'k', 109.2, null,'d',2, 'mm', '2023-10-21'), + (3, 1, 'k', 99.5, 'a', null, 1, 'yy', '2023-10-22'), + (1, 3, 'o', 99.5, 'a', 'b', null, 'yy', '2023-10-19'), + (2, 1, 'o', 109.2, 'c','d',2, null, '2023-10-18'), + (3, 2, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), + (4, 5, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-19');""" + + sql """insert into lineitem values(null, 1, 2, 3, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'),(1, null, 3, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (3, 3, null, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', '2023-10-19', 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (1, 2, 3, null, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (2, 3, 2, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', null, '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-18'), + (3, 1, 1, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (1, 3, 2, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17');""" + + createMV(""" + CREATE MATERIALIZED VIEW t_mv_mv AS select + o_orderkey, + sum(o_totalprice) as sum_total, + max(o_totalprice) as max_total, + min(o_totalprice) as min_total, + count(*) as count_all, + bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, + bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 + from orders group by o_orderkey + """) + + sql "drop view if exists t_mv_v_view" + sql """CREATE VIEW t_mv_v_view (k1, k2, k3, k4, k5, k6, v1, v2, v3, v4, v5, v6) as + select `mv_o_orderkey` as k1, `mva_SUM__``o_totalprice``` as k2, `mva_MAX__``o_totalprice``` as k3, + `mva_MIN__``o_totalprice``` as k4, `mva_SUM__CASE WHEN 1 IS NULL THEN 0 ELSE 1 END` as k5, l_orderkey, + sum(`mv_o_orderkey`) as sum_total, + max(`mv_o_orderkey`) as max_total, + min(`mv_o_orderkey`) as min_total, + count(`mva_SUM__``o_totalprice```) as count_all, + bitmap_union(to_bitmap(case when mv_o_orderkey > 1 then `mva_SUM__``o_totalprice``` else null end)) cnt_1, + bitmap_union(to_bitmap(case when mv_o_orderkey > 2 then `mva_MAX__``o_totalprice``` else null end)) as cnt_2 + from orders index t_mv_mv + left join lineitem on lineitem.l_orderkey = orders.mv_o_orderkey + group by + k1, k2, k3, k4, k5, l_orderkey, mv_o_orderkey""" + qt_create_view_from_mv "select * from t_mv_v_view order by 1" + + sql "drop view if exists v_for_alter" + sql "CREATE VIEW v_for_alter AS SELECT * FROM orders" + sql """ALTER VIEW v_for_alter as + select `mv_o_orderkey` as k1, `mva_SUM__``o_totalprice``` as k2, `mva_MAX__``o_totalprice``` as k3, + `mva_MIN__``o_totalprice``` as k4, `mva_SUM__CASE WHEN 1 IS NULL THEN 0 ELSE 1 END` as k5, l_orderkey, + sum(`mv_o_orderkey`) as sum_total, + max(`mv_o_orderkey`) as max_total, + min(`mv_o_orderkey`) as min_total, + count(`mva_SUM__``o_totalprice```) as count_all, + bitmap_union(to_bitmap(case when mv_o_orderkey > 1 then `mva_SUM__``o_totalprice``` else null end)) cnt_1, + bitmap_union(to_bitmap(case when mv_o_orderkey > 2 then `mva_MAX__``o_totalprice``` else null end)) as cnt_2 + from orders index t_mv_mv + left join lineitem on lineitem.l_orderkey = orders.mv_o_orderkey + group by + k1, k2, k3, k4, k5, l_orderkey, mv_o_orderkey""" + qt_alter_view_from_mv "select * from v_for_alter order by 1" +} \ No newline at end of file diff --git a/regression-test/suites/ddl_p0/test_create_view_nereids.groovy b/regression-test/suites/ddl_p0/create_view_nereids/test_create_view_nereids.groovy similarity index 100% rename from regression-test/suites/ddl_p0/test_create_view_nereids.groovy rename to regression-test/suites/ddl_p0/create_view_nereids/test_create_view_nereids.groovy From c770bc874866cb69a4a0bb5e04f090009e9ba149 Mon Sep 17 00:00:00 2001 From: zhangstar333 Date: Fri, 22 Nov 2024 14:21:40 +0800 Subject: [PATCH 038/110] [test](case) add insert into with tablet sink shuffle case (#44215) ### What problem does this PR solve? Problem Summary: add some test case --- .../insert_p0/test_insert_tablet_sink.out | 21 +++++ .../insert_p0/test_insert_tablet_sink.groovy | 70 +++++++++++++++ .../java/org/apache/doris/udf/SimpleDemo.java | 88 +++++++++++++++++++ .../org/apache/doris/udf/UDTFStringTest.java | 31 +++++++ 4 files changed, 210 insertions(+) create mode 100644 regression-test/data/insert_p0/test_insert_tablet_sink.out create mode 100644 regression-test/suites/insert_p0/test_insert_tablet_sink.groovy create mode 100644 samples/doris-demo/java-udf-demo/src/main/java/org/apache/doris/udf/SimpleDemo.java create mode 100644 samples/doris-demo/java-udf-demo/src/main/java/org/apache/doris/udf/UDTFStringTest.java diff --git a/regression-test/data/insert_p0/test_insert_tablet_sink.out b/regression-test/data/insert_p0/test_insert_tablet_sink.out new file mode 100644 index 000000000000000..90774639a55b70b --- /dev/null +++ b/regression-test/data/insert_p0/test_insert_tablet_sink.out @@ -0,0 +1,21 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select -- +1 170141183460469231731687303715884105727 +2 -170141183460469231731687303715884105728 +3 333 +4 444 +5 555 +6 666 + +-- !select -- +1 170141183460469231731687303715884105727 +2 -170141183460469231731687303715884105728 +3 333 +3 333 +4 444 +4 444 +5 555 +5 555 +6 666 +6 666 + diff --git a/regression-test/suites/insert_p0/test_insert_tablet_sink.groovy b/regression-test/suites/insert_p0/test_insert_tablet_sink.groovy new file mode 100644 index 000000000000000..e601362a4cde597 --- /dev/null +++ b/regression-test/suites/insert_p0/test_insert_tablet_sink.groovy @@ -0,0 +1,70 @@ +// 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_insert_tablet_sink") { + sql """drop table if exists `table_largeint`;""" + sql """drop table if exists `tmp_varchar`;""" + + sql """ + CREATE TABLE `tmp_varchar` ( + `k1` bigint(20) not NULL, + `c_varchar` varchar(65533) not NULL + ) ENGINE=OLAP + UNIQUE KEY(`k1`, c_varchar) + COMMENT 'OLAP' + AUTO PARTITION BY list (c_varchar) () + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + CREATE TABLE `table_largeint` ( + `k1` bigint(20) not NULL, + `c_largeint` largeint not NULL, + str string null + ) ENGINE=OLAP + UNIQUE KEY(`k1`, c_largeint) + COMMENT 'OLAP' + AUTO PARTITION BY list (c_largeint) () + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + + """ + + sql """insert into tmp_varchar values(1, "170141183460469231731687303715884105727")""" + sql """insert into tmp_varchar values(2, "-170141183460469231731687303715884105728")""" + sql """insert into tmp_varchar values(3,'333');""" + sql """insert into tmp_varchar values(4,'444');""" + sql """insert into tmp_varchar values(5,'555');""" + sql """insert into tmp_varchar values(6,'666');""" + + qt_select """select * from tmp_varchar order by 1;""" + + sql """ set skip_delete_bitmap = true; """ + sql """ set enable_memtable_on_sink_node = true; """ + sql """ set parallel_pipeline_task_num = 2; """ + + + sql """ insert into table_largeint select k1,c_varchar,cast(rand() * 50000000 as bigint) from tmp_varchar where k1>=3; """ + explain { + sql "insert into table_largeint select k1,c_varchar,cast(rand() * 50000000 as bigint) from tmp_varchar;" + contains "TABLET_SINK_SHUFFLE_PARTITIONED" + } + + sql """ insert into table_largeint select k1,c_varchar,cast(rand() * 50000000 as bigint) from tmp_varchar; """ + qt_select """select k1,c_largeint from table_largeint order by 1;""" +} diff --git a/samples/doris-demo/java-udf-demo/src/main/java/org/apache/doris/udf/SimpleDemo.java b/samples/doris-demo/java-udf-demo/src/main/java/org/apache/doris/udf/SimpleDemo.java new file mode 100644 index 000000000000000..0b7f51372cc4f3b --- /dev/null +++ b/samples/doris-demo/java-udf-demo/src/main/java/org/apache/doris/udf/SimpleDemo.java @@ -0,0 +1,88 @@ +// 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.udf; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.logging.Logger; + +public class SimpleDemo { + + Logger log = Logger.getLogger("SimpleDemo"); + + //Need an inner class to store data + /*required*/ + public static class State { + /*some variables if you need */ + public int sum = 0; + } + + /*required*/ + public State create() { + /* here could do some init work if needed */ + return new State(); + } + + /*required*/ + public void destroy(State state) { + /* here could do some destroy work if needed */ + } + + /*Not Required*/ + public void reset(State state) { + /*if you want this udaf function can work with window function.*/ + /*Must impl this, it will be reset to init state after calculate every window frame*/ + state.sum = 0; + } + + /*required*/ + //first argument is State, then other types your input + public void add(State state, Integer val) throws Exception { + /* here doing update work when input data*/ + if (val != null) { + state.sum += val; + } + } + + /*required*/ + public void serialize(State state, DataOutputStream out) throws IOException { + /* serialize some data into buffer */ + out.writeInt(state.sum); + } + + /*required*/ + public void deserialize(State state, DataInputStream in) throws IOException { + /* deserialize get data from buffer before you put */ + int val = in.readInt(); + state.sum = val; + } + + /*required*/ + public void merge(State state, State rhs) throws Exception { + /* merge data from state */ + state.sum += rhs.sum; + } + + /*required*/ + //return Type you defined + public Integer getValue(State state) throws Exception { + /* return finally result */ + return state.sum; + } +} \ No newline at end of file diff --git a/samples/doris-demo/java-udf-demo/src/main/java/org/apache/doris/udf/UDTFStringTest.java b/samples/doris-demo/java-udf-demo/src/main/java/org/apache/doris/udf/UDTFStringTest.java new file mode 100644 index 000000000000000..cb2eb45c9c1f922 --- /dev/null +++ b/samples/doris-demo/java-udf-demo/src/main/java/org/apache/doris/udf/UDTFStringTest.java @@ -0,0 +1,31 @@ +// 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.udf; + +import java.util.ArrayList; +import java.util.Arrays; + +public class UDTFStringTest { + public ArrayList evaluate(String value, String separator) { + if (value == null || separator == null) { + return null; + } else { + return new ArrayList<>(Arrays.asList(value.split(separator))); + } + } +} From 9fc6a5ea09660909a864c8f916b06300db7443f8 Mon Sep 17 00:00:00 2001 From: Socrates Date: Fri, 22 Nov 2024 14:37:43 +0800 Subject: [PATCH 039/110] [enhance](regression-test) fix hudi incremental query bug and add hudi p2 cases (#44003) ### What problem does this PR solve? Problem Summary: fix hudi incremental query bug and add hudi p2 cases --- .../hudi/source/COWIncrementalRelation.java | 10 +- regression-test/conf/regression-conf.groovy | 3 + .../hudi/test_hudi_incremental.out | 349 ++++++++++++++++++ .../hudi/test_hudi_schema_evolution.out | 33 ++ .../hudi/test_hudi_snapshot.out | Bin 0 -> 348526 bytes .../hudi/test_hudi_timestamp.out | 6 + .../hudi/test_hudi_timetravel.out | 236 ++++++------ .../hudi/test_hudi_catalog.groovy | 39 ++ .../hudi/test_hudi_incremental.groovy | 111 ++++++ .../hudi/test_hudi_schema_evolution.groovy | 48 +++ .../hudi/test_hudi_snapshot.groovy | 91 +++++ .../hudi/test_hudi_timestamp.groovy | 62 ++++ .../hudi/test_hudi_timetravel.groovy | 138 ++++--- 13 files changed, 929 insertions(+), 197 deletions(-) create mode 100644 regression-test/data/external_table_p2/hudi/test_hudi_incremental.out create mode 100644 regression-test/data/external_table_p2/hudi/test_hudi_schema_evolution.out create mode 100644 regression-test/data/external_table_p2/hudi/test_hudi_snapshot.out create mode 100644 regression-test/data/external_table_p2/hudi/test_hudi_timestamp.out create mode 100644 regression-test/suites/external_table_p2/hudi/test_hudi_catalog.groovy create mode 100644 regression-test/suites/external_table_p2/hudi/test_hudi_incremental.groovy create mode 100644 regression-test/suites/external_table_p2/hudi/test_hudi_schema_evolution.groovy create mode 100644 regression-test/suites/external_table_p2/hudi/test_hudi_snapshot.groovy create mode 100644 regression-test/suites/external_table_p2/hudi/test_hudi_timestamp.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java index 5e76996bb120ee6..7981a0b4f261ff2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java @@ -79,7 +79,7 @@ public COWIncrementalRelation(Map optParams, Configuration confi if (!metaClient.getTableConfig().populateMetaFields()) { throw new HoodieException("Incremental queries are not supported when meta fields are disabled"); } - HoodieInstant lastInstant = commitTimeline.lastInstant().get(); + String startInstantTime = optParams.get("hoodie.datasource.read.begin.instanttime"); if (startInstantTime == null) { throw new HoodieException("Specify the begin instant time to pull from using " @@ -89,16 +89,18 @@ public COWIncrementalRelation(Map optParams, Configuration confi startInstantTime = "000"; } String endInstantTime = optParams.getOrDefault("hoodie.datasource.read.end.instanttime", - lastInstant.getTimestamp()); + hollowCommitHandling == HollowCommitHandling.USE_TRANSITION_TIME + ? commitTimeline.lastInstant().get().getStateTransitionTime() + : commitTimeline.lastInstant().get().getTimestamp()); startInstantArchived = commitTimeline.isBeforeTimelineStarts(startInstantTime); endInstantArchived = commitTimeline.isBeforeTimelineStarts(endInstantTime); HoodieTimeline commitsTimelineToReturn; if (hollowCommitHandling == HollowCommitHandling.USE_TRANSITION_TIME) { commitsTimelineToReturn = commitTimeline.findInstantsInRangeByStateTransitionTime(startInstantTime, - lastInstant.getStateTransitionTime()); + endInstantTime); } else { - commitsTimelineToReturn = commitTimeline.findInstantsInRange(startInstantTime, lastInstant.getTimestamp()); + commitsTimelineToReturn = commitTimeline.findInstantsInRange(startInstantTime, endInstantTime); } List commitsToReturn = commitsTimelineToReturn.getInstants(); diff --git a/regression-test/conf/regression-conf.groovy b/regression-test/conf/regression-conf.groovy index 2ee288d489487d8..ab9bb0beb918697 100644 --- a/regression-test/conf/regression-conf.groovy +++ b/regression-test/conf/regression-conf.groovy @@ -205,6 +205,9 @@ extEsPort = 9200 extEsUser = "*******" extEsPassword = "***********" +enableExternalHudiTest = false +hudiEmrCatalog = "***********" + enableObjStorageTest=false enableMaxComputeTest=false aliYunAk="***********" diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_incremental.out b/regression-test/data/external_table_p2/hudi/test_hudi_incremental.out new file mode 100644 index 000000000000000..b1bdad85013bfce --- /dev/null +++ b/regression-test/data/external_table_p2/hudi/test_hudi_incremental.out @@ -0,0 +1,349 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !incremental_1_end -- +9000 + +-- !incremental_earliest_1 -- +1000 + +-- !incremental_2_end -- +8000 + +-- !incremental_earliest_2 -- +2000 + +-- !incremental_1_2 -- +1000 + +-- !incremental_3_end -- +7000 + +-- !incremental_earliest_3 -- +3000 + +-- !incremental_2_3 -- +1000 + +-- !incremental_4_end -- +6000 + +-- !incremental_earliest_4 -- +4000 + +-- !incremental_3_4 -- +1000 + +-- !incremental_5_end -- +5000 + +-- !incremental_earliest_5 -- +5000 + +-- !incremental_4_5 -- +1000 + +-- !incremental_6_end -- +4000 + +-- !incremental_earliest_6 -- +6000 + +-- !incremental_5_6 -- +1000 + +-- !incremental_7_end -- +3000 + +-- !incremental_earliest_7 -- +7000 + +-- !incremental_6_7 -- +1000 + +-- !incremental_8_end -- +2000 + +-- !incremental_earliest_8 -- +8000 + +-- !incremental_7_8 -- +1000 + +-- !incremental_9_end -- +1000 + +-- !incremental_earliest_9 -- +9000 + +-- !incremental_8_9 -- +1000 + +-- !incremental_10_end -- +0 + +-- !incremental_earliest_10 -- +10000 + +-- !incremental_9_10 -- +1000 + +-- !incremental_1_end -- +9000 + +-- !incremental_earliest_1 -- +1000 + +-- !incremental_2_end -- +8000 + +-- !incremental_earliest_2 -- +2000 + +-- !incremental_1_2 -- +1000 + +-- !incremental_3_end -- +7000 + +-- !incremental_earliest_3 -- +3000 + +-- !incremental_2_3 -- +1000 + +-- !incremental_4_end -- +6000 + +-- !incremental_earliest_4 -- +4000 + +-- !incremental_3_4 -- +1000 + +-- !incremental_5_end -- +5000 + +-- !incremental_earliest_5 -- +5000 + +-- !incremental_4_5 -- +1000 + +-- !incremental_6_end -- +4000 + +-- !incremental_earliest_6 -- +6000 + +-- !incremental_5_6 -- +1000 + +-- !incremental_7_end -- +3000 + +-- !incremental_earliest_7 -- +7000 + +-- !incremental_6_7 -- +1000 + +-- !incremental_8_end -- +2000 + +-- !incremental_earliest_8 -- +8000 + +-- !incremental_7_8 -- +1000 + +-- !incremental_9_end -- +1000 + +-- !incremental_earliest_9 -- +9000 + +-- !incremental_8_9 -- +1000 + +-- !incremental_10_end -- +0 + +-- !incremental_earliest_10 -- +10000 + +-- !incremental_9_10 -- +1000 + +-- !incremental_1_end -- +9000 + +-- !incremental_earliest_1 -- +1000 + +-- !incremental_2_end -- +8000 + +-- !incremental_earliest_2 -- +2000 + +-- !incremental_1_2 -- +1000 + +-- !incremental_3_end -- +7000 + +-- !incremental_earliest_3 -- +3000 + +-- !incremental_2_3 -- +1000 + +-- !incremental_4_end -- +6000 + +-- !incremental_earliest_4 -- +4000 + +-- !incremental_3_4 -- +1000 + +-- !incremental_5_end -- +5000 + +-- !incremental_earliest_5 -- +5000 + +-- !incremental_4_5 -- +1000 + +-- !incremental_6_end -- +4000 + +-- !incremental_earliest_6 -- +6000 + +-- !incremental_5_6 -- +1000 + +-- !incremental_7_end -- +3000 + +-- !incremental_earliest_7 -- +7000 + +-- !incremental_6_7 -- +1000 + +-- !incremental_8_end -- +2000 + +-- !incremental_earliest_8 -- +8000 + +-- !incremental_7_8 -- +1000 + +-- !incremental_9_end -- +1000 + +-- !incremental_earliest_9 -- +9000 + +-- !incremental_8_9 -- +1000 + +-- !incremental_10_end -- +0 + +-- !incremental_earliest_10 -- +10000 + +-- !incremental_9_10 -- +1000 + +-- !incremental_1_end -- +9000 + +-- !incremental_earliest_1 -- +1000 + +-- !incremental_2_end -- +8000 + +-- !incremental_earliest_2 -- +2000 + +-- !incremental_1_2 -- +1000 + +-- !incremental_3_end -- +7000 + +-- !incremental_earliest_3 -- +3000 + +-- !incremental_2_3 -- +1000 + +-- !incremental_4_end -- +6000 + +-- !incremental_earliest_4 -- +4000 + +-- !incremental_3_4 -- +1000 + +-- !incremental_5_end -- +5000 + +-- !incremental_earliest_5 -- +5000 + +-- !incremental_4_5 -- +1000 + +-- !incremental_6_end -- +4000 + +-- !incremental_earliest_6 -- +6000 + +-- !incremental_5_6 -- +1000 + +-- !incremental_7_end -- +3000 + +-- !incremental_earliest_7 -- +7000 + +-- !incremental_6_7 -- +1000 + +-- !incremental_8_end -- +2000 + +-- !incremental_earliest_8 -- +8000 + +-- !incremental_7_8 -- +1000 + +-- !incremental_9_end -- +1000 + +-- !incremental_earliest_9 -- +9000 + +-- !incremental_8_9 -- +1000 + +-- !incremental_10_end -- +0 + +-- !incremental_earliest_10 -- +10000 + +-- !incremental_9_10 -- +1000 + diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_schema_evolution.out b/regression-test/data/external_table_p2/hudi/test_hudi_schema_evolution.out new file mode 100644 index 000000000000000..12dd0cf086d3f04 --- /dev/null +++ b/regression-test/data/external_table_p2/hudi/test_hudi_schema_evolution.out @@ -0,0 +1,33 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !adding_simple_columns_table -- +20241118012126237 20241118012126237_0_1 1 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 1 Alice \N +20241118012126237 20241118012126237_0_0 2 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 2 Bob \N +20241118012126237 20241118012126237_0_2 3 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 3 Cathy \N +20241118012132306 20241118012132306_0_3 4 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 4 David 25 +20241118012132306 20241118012132306_0_4 5 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 5 Eva 30 +20241118012132306 20241118012132306_0_5 6 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 6 Frank 28 + +-- !altering_simple_columns_table -- +20241118012136512 20241118012136512_0_0 1 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 1 Alice 25.0 +20241118012136512 20241118012136512_0_2 2 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 2 Bob 30.0 +20241118012136512 20241118012136512_0_1 3 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 3 Cathy 28.0 +20241118012138287 20241118012138287_0_3 4 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 4 David 26.0 +20241118012138287 20241118012138287_0_4 5 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 5 Eva 31.5 +20241118012138287 20241118012138287_0_5 6 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 6 Frank 29.2 + +-- !adding_complex_columns_table -- +20241118012144831 20241118012144831_0_1 1 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 1 Alice {"age":25, "address":"Guangzhou", "email":null} +20241118012144831 20241118012144831_0_0 2 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 2 Bob {"age":30, "address":"Shanghai", "email":null} +20241118012144831 20241118012144831_0_2 3 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 3 Cathy {"age":28, "address":"Beijing", "email":null} +20241118012146150 20241118012146150_0_3 4 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 4 David {"age":25, "address":"Shenzhen", "email":"david@example.com"} +20241118012146150 20241118012146150_0_4 5 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 5 Eva {"age":30, "address":"Chengdu", "email":"eva@example.com"} +20241118012146150 20241118012146150_0_5 6 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 6 Frank {"age":28, "address":"Wuhan", "email":"frank@example.com"} + +-- !altering_complex_columns_table -- +20241118012147879 20241118012147879_0_0 1 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 1 Alice {"age":25, "address":"Guangzhou"} +20241118012147879 20241118012147879_0_2 2 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 2 Bob {"age":30, "address":"Shanghai"} +20241118012147879 20241118012147879_0_1 3 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 3 Cathy {"age":28, "address":"Beijing"} +20241118012149007 20241118012149007_0_3 4 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 4 David {"age":26, "address":"Shenzhen"} +20241118012149007 20241118012149007_0_4 5 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 5 Eva {"age":31.5, "address":"Chengdu"} +20241118012149007 20241118012149007_0_5 6 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 6 Frank {"age":29.2, "address":"Wuhan"} + diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_snapshot.out b/regression-test/data/external_table_p2/hudi/test_hudi_snapshot.out new file mode 100644 index 0000000000000000000000000000000000000000..efad67ffbfa8c407fa42d0dd28f569f492bd1d12 GIT binary patch literal 348526 zcmeFa`IBTCE>vSMpe&OSk|{bui)n@^%lgTt!VAMR8MeYQBV;qoMU>1f6hq5&Savwj_i~ZyMT;^z25e%{r+UT><@}JRHS+C_98!YdY)g{y|}ckp5-}t7~64}Bw5dT!|FMf*K_>n z4fFkMJGShd!({x{v^ZMt4Sn13J$K7>)Ccd`cF*>1+3v3E>{{-QYsYSAZI#JjQf#?9 zLA>?<{q{fR5+@ju+>L&= zpN#X#k$xrjM+Hyc?oZ@j><{ICKOFJ&;Ejxbm0!7m8+-a+9gXw;kpE$FZugyY12);w zR9@1v{T;q_+Yb4k2czNM>GR=EX zTQ-Sl54R@uN!8Fy%YFxY?z_y8O)FyMO$UT)=P^V`w?e7FDRm3YPpz`JaE~pX>42m znOdQ>-Z0=(+tF6wE?6PQ+qL{1KfuPil@)S!d{?Z{dtAjt=u7{(|N9^Od-~yJ?t+Ay z&pof7_}#z3{lj0-KRm4OiqCO>^~ZnvpZ^;7-_y_O+aG@Qa%FC!o~6yr$v~TzecVSe z+*3b{`+NIG%Fu{8>1WeHG9JkG#e0lR<8pH}J=)*CTHMoiWM*8hCc{ahwnE*qa6G+G z?T}H2WeJ=e$MqeX{aLpw!H&Z(I?amhg>g~@Hb*6ldR~&ny*P}mUKoWzII<_M=KAF!zbymEOwgOUY-TLJhWwzoS+vaLDUPpAn;wgEV8^|StbRhudzbo zLYx8q13N0Kw>LXOqd%_Kndn z(I2(GYOZIydMkGuW_G(6=63H@mUixizC?G#a$aU9y(F{o?@my9rDp|2*jTm0@#tQk zxPrwrJy@Cy@0tq~VJ(+}sC1q6lft#nPx45H{@n|}1?5X_%yDTpqm1RHcC+bw} zO>2(c4=ik^{1weuNZTxZJl5S$*RpId9T(e|lW{s4n>*@4G9LGfY`>UP#KU9pVSuxh2Tl;!F-z{INze0e^idIWK;6;~oUC+{C_htg zzt$T@4u-+os*>Ck^hDLGV|i!DefhC(mV$-NtGi{XI);r7y3Jv=Y^5Wt7?ASc6aQ+bL>t>UdCnM z=SkQra8f-lbbV69-0$&aCQeCkhpUqJHmz=z^Ugwa!}d-_2L)N}aL=?O1c5tlcLeOS zpEj~1=Ld2Gw=Yb`_e^iDW20-yxZkpdtf05bJJ~&lmn@zMHqgSEkY_j(p-Tc8r(Pa8 zq|_nFG&UGJ69@P(oD0Vyx6_zCcb0khT5lLcR^$g;u3d|OB?;g4qaBaf-1X;qxUCMw z6We-+A9MG4u6taogYZxOCXc_tmCN;a_5J5BzoQ>;I}GO!tFwWLusItTOYMkpIERJg z$#Ou%Ws_qPdC9&yBBtQS!%-hMZKTmJBzK8F`h!6~IhX|;vx9OQ!?*oPf4tWpwtd$Y zaYV56ym;156#20o`U~epqRz;T{jO{}4lOUV?6^meoA$goAplOZ94F+KmYdj~TdH?9 zPs@Q=>qH0orn?=G>&mCx9})U+bmSxmw%^VTo+5%}&d~$@cZS%e_*R<+wP( z>+bE#C#bs*#|^@Gh5OI}_~KDn0$D}&mK!Cx>x5x1jZ(MgrFqhe<)*YU5D+iUEt0mg zJSuCwq3Z=9pp>JN+?o5ZckOuB33ehc41;iyT?Kh6}Lpdh`RBgN5JoQf{lQb_MFn@T1A0n@R{ zn(gzG{XwCjpVdf~9s71HZ`JLoyl5wi-Pmn-Dt5i?yH?nB^pe6N!7MS-Y!{*rP=_Q- z!=4)lZkChqTWKSiO0tY-PT3jrZHn5>T5+<6Ob119?RCM1=8{({#WIwG}g|hm1hLf3O#2l!*e=8Uh)@&Imo)} z^Q<&pc`KlS?XkN*4jiPT+^)X6|M1V~2lrmp_aFZX_fP#$f`n(P zXSn;iemKAV$}@t>e`)wZ{mlHcsz^C6Qo)Ah3nMPb)Tu0|N2EAvmnGB6bR`3cn+5%n z^j%r5qcKpYM3uYeK#ffs^_}8)`-LLepJkA<5aIdJaZ42FM%a4wZk>^|7w=Gbu`O>Q zG@!DvW5vE7MO_9e@PODoH|~W&QsCb*{2NIoKZ3nFnP-RO%l!4`*#n!X4w(@|A7jk* zvn4RL85_iLv@9bR#C@aKfMEHqtn5k&0V|Co68{PK77{(oNIaa$d$$9Y&C3ae+wc=ED8Q6X3|&F5BlFw#yV z$qz^!C1gI4f{+}WDvo<=nhdmol)@BsC?jyzY0%AVbaZqeXt8X$*{RU}=VEeqbkuS> z9NV^*pN_!a5g+X=#?5vZ?trROQtUb$d6LI2HBG)?08lT><6fFtaj*1KugEgTO3Ox? zIW7)oajS7U_9(7ttB_4=fwN0$<8;=dHplb6Qw%6NZQmp=AMTk4M6d8^U{v=@pp;T< zTI!dUqE=}Zcp1gqUgm(ccpRNx90z!zpmbd)u{=t+i*|&Fu4pkNeWF6cX-C#h*zL2i z9y;t_*JqzUNe0RGtzrzSuWh*AlPYd}_MlG%gH@=nPGpCbjauflmsIQ0ggSMZdcD+- zOWd2rzb%?KwMfH>4A83=K=1HvlD{>Ik7)teY6X z{W(rnUo7(XXccmfD`HD`ELM>DS&`V{paY;AioLxw4DwzXc)4$TR+QOlSDTA9p&qYx zwYgZt$?Anvv!{caN%E-uE?FiG!m`Zkp13Qc{o(dCNof{Jzvn3=4mx0TlC5wR^u?|M zF|PS+$5I3k!)I5|;IrcpoH$6_2?|S5To*%`QhJ&4*z?L|f%l6AJ=m*@M+!>FYCvw5RxV#>YALH)$wf7`$ z{-eLn&0|AuxZK>)_n+2xALdSQ04{gNZ~y*(bW=a(_RY_7BPC*e`O@S1qv4S(2?0{I zp_1Wg!ZkPM$)0@AItJ(;B;Z|wmFL2jWoy;Jnng74Y zCOPI`Z`x(ci!9|;Iy3jc-U)q|sz3YHgqYoECkTSbcDr1^%fYLlkoTH9}$n z^A0;i_>}QEI=HWk}=>sA4P4^`clDc3v`0A)V~P`-|pW^*Up^;yfxq+ zcq8?DQI-|GC~^}kN|GR^s>S{^O?HyvR2G*?9V~NsLM-)?Vleg8&0?cl;uIhp+}5Ij#BkH2hng!s_0I)3DMiCd6>LZ9eS_R4#yO+E?=Rz!{!#A)O$ zu12elNR;V;JwjHM%mEM?N@Jup>k+FRlfZq(QRp@x7e{gWpS>3dv zz+c76z@147+P;Y*yNELhRVGYaBL?DYES(t#lFIGwQ)ab@+bZxRGZV$x(9TuX*il9tx97db?iD|Q#*Qj-A& z0WQ>aq&ZwB&kzNGg+*I1Gr}T-U79;k=saJ+Wx$SEnHQlG*-@0HXBq{p^@fyiTuM-0 ztpH{Pga9CB)P5q%n-Ni=fGHwsS$Sn*S0Aq|hvrb(4RBE5KuR8-+#4xUrD6f-2$|C} z=-(H&JWhnoWjJA-CtiV5IhiHyGe`WwXxg7ZJZj0p11eCJ}+nQW4eu+OQQ67ngZy%5Iyj;}wFv zeEY%}L}@&#sLMJ@e})P~r;UX#;H+}g6@#SR#^xjlu@%R?D9p&xas0hBwg56*7s6;_ z=Rw+7LzQDLxFLYwdl0XAcF`gEp-tQ=R<@%6A)KiVcHHg#q8PB@F2RigZIjQT7wS@y zLbe&S#(sexR7AvOF58kE)TGtItLbJHK;qc)OPgA0QS__?A~NR`rj1J(CrJuBDzYMV z9Oq1p>RNB;5xzyvR-ow+GdR@Wb%R~UgMAp)aFW`W8LVY+;+ zUC=*PmA(riYrt_QPW@?&p-py zs;WQ6um&vmPShQA#hVt{p2bR$SOH4S6NPG%SHLj8R=p4lfhob`YKyuAX$i{+JirFR zjH&lyQ5_5c6tXN+EaEKi6IZ?(bAb+v(P71uHgKs0=F$FWRE4hW1X5)XM|1h&c*pM` zHy zw@SKYN@xa@a{|6F7w>hZ{%Nf@bYN}~@dv6#&U*7#`nTZmxYn-p;DkU$74%0hTCnx8 z?^YNe9_SNukHmv7{K>CB&E1FIsA}Il&bSJ*=fdEppXKh|2e~=n`hBkUB|!kik8zQP zwR2p5=WpfuiSW+w>8eb=BSm;FmAeacM@aED@SQG|9FMVpva=wl%#fT+x`}XwrLdBP zHm5m{pm^dJ30dxuNL0O2G>eG0JWK>Us}%(+%WrI&StfFeb|P9Mv^}VwJ;bqgWNDCTSBO4AAhFNptD9x~89NLCQkPwSVtc9wg z7R5M+k=so;)#q?C;`({jlt#*ZgOjIisg4b2ZZjFqORe=Kw4G2o9$3Tf0FkV_8yw6_ zax+-CJql-G&nw~raIg?cN|`2QW|eW%;2&vnmuc@4(!`pnvMH8^-M*2}v>6w##0S2UZqm zK~6KkncAkc-Y~K(7qrAxJ!#rD1-C3`*H=AhHt0LW2*pzOD*BGL6=HZ^=K2S}Af&>_ zxqjwkC{1bp6W`)iUyh|6?cRs8s^sv!io3&;5^>*u`n=E&K2RAX9$tN{$1Ru7=sRxT z`p}>LeeO<8lW^NUax z&34aC?t=4~(1_4>07m-7tn+GSj)WW6XU{Yv0oV7~FH5bFgv(H&EJdF+fDwtKN!!Ct zT5puNK7BZL4-TpzrzHPzSV95wA$L1YX%ThFTbluDYu!}1yu41@_7nimV_JUq2xJZO z-Czmj+Rft9@33Yg8p4`j{0oE8N!}Ta+AdJ%3gbkK*A2&q8R&Zmf|*1ZaMD8_ z6~2%UZ7X8^T)SvC0ji}@-=h)NPm1xqSyVM5g)^7SbCy&1H)v#ek@4eAf&^ zWl!ErK$VX6TVOe10<8+&S*-5bd`!1w4p!O=X_Ui~M?Ss#)ST(U_w1C)n&T?pZ|*UI zazL;)mv?Y7+h6B_n>mQJRwVL_W#EXfJdW(mHQTb!%c-Y%QXNYZgu@}~dU)OK!LtKH zU+(snE>`Mhr4y$m&C#SdUdSUn9@QQREenX`S+f@~rQ*4`ujXm?DQk$QYHKCLzPKZ3 zCri{6f(_Q{=8-^EkbEcexJ(BN3bpJ=b9}bfsx*fM>jo|D-1j(otdMI_)bO&b7X<>i z<(NaO^fTXX6j_Bdw76h{(Wuq~NZAO!Hh+FgN+W>BrnYL<(XoA@KhCQBUN6s;g!W0* z;o4T8h8yl_tJob#u|O@+LO0tNGr#2o(4}ef?`3q2gWZJm?>Hd7R_f;fZ9$S2XWPHC z)*Dh5kN9vf1U2oSOonKIP{G~>Xb%auU2oRYhSwOzb3eBvU%;qtyq`o41ZPS1&O_Sdh;Ua0k-%g_7Juf@$v4l*td09ho%7>^Wg zVBm35$(v~p%$)fv#rObNLH)b6&d%79?an{nnOWEo4Q(O1CX0!MPm^T?qqeiektG4G z5`GV!A+B5~hol&B&*#6wNxhJUHhE{$%;ucf!RT<-DQpJLd%CS&>?{9MF}&9wkA{mJ zgvh3!sN11SxTlQay5{yO0LcjrPDwu1VV7xgZ_-4s>|EXC25un5JNb~yWH7odNKsgU z7bYr`m(~GlqX~piQapNK+{{H)ra?n!ZNDy*NC#CjMNl}#%h%jFVkbLx_hKO#vKdIC z2^%nCsmH1HQp5%&lRw}9Gr_s7OwxE1?W?prWhq_-Gw4mFIVNYY;6 zX0&6Vg<(f$OJZK@4JqXy$>S&v^E~m8tOHzRCy20i;S98oqlyoOeR%p^u@d=Lw5j0s zv1cBBp1Xhg8~T~o=D*;<6|R5ytK7UUP)KE0B(If1Dm)R5S(s5Va2kd|jLKm`T8~X( zFX^t?{(fiF3osqJ?x8c&|Cu+amZu#MrinQm(a)+JaSEuzr`1F-BVnM_ zyG+~x9@%F01TmZn)0l_eIr z0rpQSzLAkzC1cGL+fVvxr6e&n^D79E*n!TKg14vHqwB8J`QdQ;1@u-7W2{2oE6L;l z0^_V=S9f0|-)q?9B{~==H36YO4a?*d5&+XY;zk*)9&i^_uBb*ga?zs}5?wJ=u)1;j zAkNv{ta*xX_|y$H(v2d8@kjgm=z!qJSR{S$uHwg9&=2^Nu702H%~T}_b)U6_#2Y}8 z1z1PrhekJt8XO8vQfbZ$PNO8KSy(tl8J{Uyw$>Z^Uf|Ldr0F|`(?Y??vv<(sqMAEL z%lxgcj_^8MUKZ6jE;m2Wd$0eakHi;z>0tjFSHbi3NWl7M(=yS^IQ7iPjM&O za5o+nsQxb3ceoA&cpJTs8!k^gJ(i|lF5(5beCx&ockVnWg@6Z69iD^31MJd}J&sDX zE`o?nXQfQ6>FzmDG=bHO==pX5&tW`gy3YdWB{U(Sswp;KX9(?BJLoWlK_i65v%zlf zRD`hOq=eB+THyPb9VTHLwX%7)#^Mu`T?FJN#8&XnaW z1;MXThfoFFQMwiFhMIo&&CVBw`eaFkRy9V=c+m?sh!Aa6Tu4x}1P`;F)Un|J9uVmYlOmW-8qYs%W1CR*gMmSo{b6>Wyr-=f2By#4IkM+J(6 zH6wI!IwaRI>O}P=a!4|?H|RD)EKxTKM%PB3LbjzXS-NW-jE?ja_EJC{J0r0xv&Iss z1I30jgQkjn*-!hb5za8zu>6;#7s9#U$Xs6;WyRnGJcpjt(Xr|Ml|scWvauP4U0*6N zIPvX#?qy4*`jgw;f0RZ}O1l#+bDmN!qM65{Q-!()J?{>1V$uK%b2zE8cNSMmY0k*A zN26Rv2rBBV(Is3|G5l^IEb7T;<)2tL4tNptayS~}=qO8?3s+Um_$}P*jw&{U!gadn zB%Q7MyJfnmvA0DCp4%}RP=1E1^Fs8+QWrhB(i6!-LIvMC+v?C-Z-{^wsmd0jEGFI) z?Aqv1I7A;x&84aiA^tj&!gK}Zh@+B{w7&ezfcwwQe|-8|?>+eDThg@sA^n`Y8|w%C zpLt81W!1~ApX0Vto6tXh;in$rLFGC>qMv)5yYGMFv)_7IbYjfQjqg@oka*F<3s!Ly z`-N;SO;TMUqNS6QD~fEA`en(%VIulTyXQE^gUS*iYIVpng!_BrRBRv0%Hq1#oWt(Zp06jPR!+@=)Cil|6>V6v3?%0$QmiwjjsJ1dK4`hm&5 zGIppgLj*It0AJIx8XEww{`l(}GbOo+@TR#w*OMbEFEqQf>Qls^kQ&KaX8)y|e!+q* ziP%Gw9bia+NhdLaUDD-YJG3v*JWu108)vTT=eUhCjo8&-i3jK@O~iGTJz(~NvTD%M<1OW04nC*^K^Irk z{`eQK+?VdU$~x&+eS3TN*&hiJ=l#p&P!?)1LFirW1iBLZIA0oFE)$oFVZeiXklnGn0OX< zF=@wng@e8drK*^4?GczzD=9!C1#K13u+hcpB{A)0DZ1%l0`=R88<`M%WZf8IE?q^N zV72z9wU~==qf_UW!Zy|6&CHbOXO9zjN(YB(KWQvncU6&rQOW5DR&C2&9jq>i>yZ&m zY5oa%bp67C2Kl1%T(lu+tESl|V6!RPZSJyEYK;qRwMy;zjo%jm(06Vx6& z7Q^g(cZeOjXwbesC=i_7X}Y*|t=qJTK{4MM6tcf|6l&wvu~my&;`uflP@AYOw-x zt?@3*s91@h&Coqs58n|!t)O#Nf`MI>Hj%-+MO^4Yyb9n{v{$Q?0n;K(*m_6aEBy>Y zTR%ChbY*5Zo1&vJcR+`C^*r8YbjI=`5+BN#|NGA~-K#jMvSWWc6{0;;b>!H*J3)u=1DWf(M4{G;R1;`Yw8jFhpfLqA@kWg`7}+A*$s{waLmd@oi#=M`bY zsQA~HjLOGK;bt7KB=7B((aT)V_wcmRKFYel%uA4(=q0pGrSM_FARB5ybm%r6?_p7~ z_La!pk!X$@tHhju9j_bl*tpy7YX6{3^-okg@FE?WFNA)p(gcn^cC=%yl7I;>LFV;P zhy%e-VZPEApZB5|l~<<-Gx1Vb7Sr=WfYY@gGe{X!5N$TKFhr6J z>Qtk&2BV?5Xs2A)QqOQ;*{UGLqj01UgjUZY?MItcZEsSGB|&d!B&yJd$A#WS0uGrH2MVWlFX*c$W@heq_@NTTHY%7VQ*;B>-pjv;kBC zGAwMd82CljvDw+FW0%c5@^u-QGTftw5GbXV>5H4C2^VwIUTsigi`H!ChfIrG^YtN(BbU@W;Rgzq;ZHY z5JaaC*;HBvWf|e<3ZDrDO;aQ_BC3Lm=4d!o%?zwvDq{TA*cJ8wT~tPtz&En23ztK( zS&n|0-Yn8|=_5rRK*3Biejf@F?NA0C4C@eXmY8rR0a5pW4?#l_28W?EXyt%TXITsv z2WC3e9ah4O64C-mgGMe@3FqC-QIL#NlA_>c`we83XVeb zOVLZSX=pxEv%S_E;@=@@lI)pgo2h#?1tbxDqUYG3a!^aE}^MOACi(A@#o0a1%76pyD|G!aOXoDE0qzj+6Mf!!YK4c|y3k6AzaF81kfLlS1v@?IBX@cmb zAIJnmo;Lj(GCaV5#sYZ`LW>;>q+`a-z~*aA5JX+3Nz#&ycM&Yhz)JWG?Am4Qz(y+8J1L{x)aYut+jg z?Lr-GR+!+dHYVR(-8Ktww+d<|^X@Jk+6m2+2)6gEoGs)*>(>A(MX>-}AS#ork??lx zqdhVza~lN^`PGcgcJBokxS995jC=_~AKT9{y-n?5?YqmRM!_uYE?KJqe4CUN}b@YsL+m&0SLp|XEDJoYb#$5y$=Sq+b^Op;eONlPPZ zbOg7`n08Mg&ZN@1?;*vAQ76WBi@-0zFsYTD?L6zX-Y`Op&vCby00ba{!@&XAw`jtX zIan28>&RMWHf`M*N`TvcYVzlQjr*V8;|7TaZn^xw|H&^&%>2aFJ9nyJS?I)% zJ;2T6XSvar0e2tz>SdW1^xi|Z0OQT=>R{_sH+p?~@HhpUeg0O1obkCVKw>H+X{ z_NG;eG#N3{g=t$jk>i9B0^A=|?Wm2on%b)TzSM*x1V%>3W}tmVAd}Mpsk1=)#k&Kf zOb=RuY==?fDn98(N1*#go+nD{1~_7~ph)KjZfBD|y&<7rlnDVNv?}TY)JfPHJ!z4k zL__t&ZN!f=xfC;aJg5ke)0Dm&`P&PbIb#HqwoKsaEDKPS-;fA-qhS8~4Q7_fo6U|} zF@z>Q=|F559!G&=tstPifP#RqTTr=z*C=b1i4q^-chAllUqd@u<~q%8h}n=8`IIJ@ zR25BmS|-{{HD3M@>5McM4T4HH?N*2ko1shdDc8@@$5&T>ln|Vf^ZvAD_eBBnc`Fu9 zP)!v*$(f#sigeGpW=kS@mq!V+-RRvH0W%vBsmP{!5W?}L(F?F4I9s@Otv3V{!ZJ(GnXZ{3>)o&_*?_%?rh3fYu`D znC!5bvAG5gpi;nkBEgqOSwr5jlQk8fUqTWpf9pARMCJv@&v|tkbr$K9o#B{0>SA9GL(%VMh_c+^};+qL{s|kUQH|PJHW5q z&DB65`fphme!amw*YODWm9wb6IK8_rj)W`G;k*cn7+1U8d9ky)5fDUDsDLqzwxHQa zIQqhnBKCqo9m|R_?XQ7VXH!jhGftjFLD0t7SE%x65!t*y$tVHV`q5$FbvZ7D9Bu0C zX?j9j=H@;XD2?n@2eLTsckmQNG+TSe?v$Se6AFM?WMqCq;KzE-Ma_+gG`FZ*mlUc{ zws4~YZi!u@Y+(gb0B{g1EnvaBXUqLr>kW~phwL=hkgXzWbgWN5Yp??z{w!UHTw8;iRx%yb?V401;awe)W((!xFLVr%-+B= zl&KOolEZ$DXQ=Rg4L7Rguc?V33goWh8FGHQi;DCdAZIC=1s&!@%hXCfJWxYip|;ED<6Nh-{YBHHz>UFs|7 zpV5CXgF6s26tF^H&dH?F4~see%scny&&JP6n!v?>_(MO)ZT(mPkJkhNkq2D9#&vJ| z4}V+##O=TOjlavSSbSlDJk9mim$}iGlsm~%st+h*Suol1)toAWLIYJjq3nx!>zNpF z%KCkgoPr(FAEFj@@;2S(JAleWdtsnU^MZ_zjuhc9pAQoH;x=;rcIrUAoJ-2#M|cTpth)!{M=Dx)(xu`9aLh)px$) zbI0XfuHUNy(%<`g{P4x^iwg8RT=nJgsz>)*PtKp3NmZQ2oPy#qjY*!3)&i!6lwOM zEzzU=isCN5P4-Y06iDfn4PX|DYV`ip=P~TmJfoB*b>j^Bw6k=N=0xUj!}07lU>1v7 zv1kU#lIRre3O=BnwuauGWcW)@9}#AE%^KQYNJl#$fG@^ers2W-7mZ@3$z(YPuqiM* zLp_c4sRW{whhydtpNH;4GEH->7`3;|UTv)Xc~PbaZeKwcLOhFUvvlCZgcfAF_$YOQ zh2BNgh1G4aSTfQBmbJ~YyFhF}BN4mG5g8JoB_M262pl`kVka-Wq9Np07RQTQDjQ7) zHN6p-TgxWbJL)s>=d2r3KEtj-CBle>=PNDK@2#L^AvL9qOMyz@1oZ5jA+ne@WX?KEBeigttSRqt?Kf&CKNM9Eie> z+8k_R2u`YoG1K0c@NfduB`JQ@jgS7B1X5yng==IOUTKq8|AhQ&vjl&MQV)N6xV(V~ zG8D5lQC3;(?ghWL(0;Hwn|mQ_Gm(m~&o5*KEAb>9aysP?>}Z9;46pryrVkjNMgV|CzME`eat z>e-AgtY#o9vFe#asJxXbfwhO#0E7Q_5NrWK7Bq`l$X6tSGyPfXHk}iB^L$mI=Z2ov=Hu-gqGCGhTSuKiDuoL-k1gtc%A;VoI@QemZ^h;igHXELTZy||Z{AtO(j3x!zDWGa&*d46;#U+Z)z zUo<~1R9a~}Uq<7@Yiw2(As8C=naWewHdM|XEiHtz0YUi`^<`rs*1Bxay8l{|kGS}U zl$;G6T52REn&0SxSmbkv^psgZKb7O<@ZirhA6e@Sp)3*tE+!<1^B|%sdKTp3l^(Oq zLE9=`yd*`_;xQ?rCZ}C8&S&_UOJ6(|#RcK<5&a*l5JmO_&Ziosa5#edtoF!!vK23& zekG_P-U>aO(cVxGo*bzRSE+deNh>pD`m5`s(V&>wiJ8AT&$Mu~?wen1%t$*|y>Vw% zmD%rti7u++^nf(bYhEIbgQOF#Q1!H6R2UG-d|GUbSSt%*g)(l6ex^`cRq|k-6+zwA z$(T%;(m7F4mtI7}zJS^8_#O5|t-Vl{S<`jnyc?r@+~1qhw(-I!(JQVE@iR0iz|$w8 zZ|Q+YV0UHaQ049I<^>_fu(XhKw=lgzofg(KEeDS1H5URAd5thwBstVqXrvB{Gjf=4 z&#(+R*Uc_F6`^eitv*u>-(;Z2uraI(wuTsB-3wngBAoMhby_9k9r&W!as&@LIgu?f zuBsHK%z8;Z;@$w^jx@$?`F5j7zb6GiE#WOk&R8No{Zz+|r=2hDWe>8!Wn((SP0Jq7 zLDilLod29u4fS`YH`n?G6Amt(DfV=f)~j+)_YjEqqLzj+}2ZjQd%ahpbsA|NLNVhPXXo74-Y z8v45EL7qb*vSPh-Uv9^S%V-cn=!9)%Lpw*37v;i;^`ar20n0%GAx(iS=6az>F)Kuy zOD0LY88!kE?>OBPnNU60EN(nM9*<5&i&8MU-!VDv=tgHZtPu?fLuG@s6wxXiD19(& zF<41djr@>UyYR{$yefLna_KugTerT}8v-4BF~yk0v=j!F!+0zU9hTF2N4$0L6WV?2 z+xG+v)RzxcYE>WR$H%H6OgxnO$o`|>|KES@zt%643dnTq=^rjjDMJ{dpOX>?mxAjp zu2T8<;U5TrRGzuP^^+g}ga7tf?&bd!a8W8JZw^!&m(;oyL6m!adGOtfpZ*dLBd(`a z5>ezkogvO*irWdk zR5e=5d8}Ki5KhR7-YZJoghjAr8l`Sw1~53dQ;VHR5kcfhb0h6CjMic1eMIQpV?1V% zLUnXXzX3qfCDQPbT9ls8j)oEaKT1518QlX$VANSQw25^O7F9UPMxlikL6|IZxSpYy zJZuLf9hYeWD-0Q>qHgvUUeiKm-R~iTAOZ+*}`X-F7g%4U&6IFx@5|y@&KWMZo>7dyA?f`?Ksy@>~ zi(ckJ5Aa-RH>NbfWCLbApqUjys9jv9$q{gd@A^bl?ZES?Z(XeOD3K%G5n%d|V zW-`Hcam`LcFLP@lQ!=bmv*{jRHrNVqWi!6X&ErI(E4*ZVu%p&ue>P{#X5M}RLmC!V zhhBXmxj;q-2Ol>g1@zo+W_0)*D8I1(f3$ zuP8`x)d(Ri#vWb9ge_8xu4r2jmK1HP@;^Gw6^8IW0S^vEyp2lWbniG*KBf~i401DN zow2`XJ9Oq@mWe0^%pYG~jgQ=&*a6`48V&|>PKdffckh9lXJ{{nQho3!MzD|3b;ZN@ zg1)*;`A(X$1M`x%xuy4MufVL%{$?IlNULEEXvpszCnR=r({A4@TKP|5h{`VMNcq-u z3@p~JH(*8RR;@4spclMkAr4#u@7OfZCj@-R5@kJxy#fIg6vLR!!jDVc;hUf1hRgf!>U(Z4KmX7NpS?4-z9rxf zm-yoQf|tHf6^Nyl{Ntbd*#{(sbfv;91r77^>_hzYhWv)h;lcb@6;?!wCY53%nV4zr zn*|Tfm`?p?0~K{_H)6K7BEATiV#IMI?a1QHCUVS9nK%uw+!pB*GDT1mHf<;41#3VN z2^AS$RO7oKk`@3OS>1F+90kfZr5k5Wdg3asWi-5x7Q;l-&WwYo_E>dD<>~tVg2%X$ONEo|<_i<;z@Vlr!BV=wo=qpB zOSIhwGxsr`e6(^BTy9k^iytSi};?8}xS-F+G^5|%YlAO{X zGIyEKtgjb)dnCL%wXZK6=Z0RDxw_B^grW| z5-p;TIg>I719gS4X zPS|NTMLkx-jq4T}L_zXmUwLc!Ksr!U8SX_iZHfWNjXHGo?}hQ<{wf^3`d-=%fR76a zZRBdgNHpaUGewQR1m!FcA;B;>{IIxex+h{`+4iUCPYa~P^OX#uLvYC64*Yq5nn{~|q*}e2WU87%+m{EqT8m~wmJoRnGFl(s8^WAI}wjM@!6#~7-PI$%IyOIwDe5uj|n_LB;};TA+i zkOdic;N{ThnX&Hl5S((YoXHb**3b*Uu{H?22q{)WT159q**OOOa9nC#V*sbiYR?;^ zc!&hj0KGU+T>Au(cx;bmeEL*ZM>FKRf^l$OGvv1bF5sk)ol1=q2dn9a#B`j{nZ zKd%sfnWcyZdjO^m*MWZ{u|PmlRN)Yd^K%5O7M({$;#fGX^_@m0s80 zT^mX39a!k?N-W2 zmZOS@1~nc*ZzCAAHu9H#((n=$fkqC2=|<-Y#XP@<0(fn2K_qj!yFRfdo1Icd>L!a; zP0fsJSH^IKkG|5PsbvzsfYc|(U(-N5O9}%CNevb)1A`)Kjf?ivnfoaMR|q?DD`m2j z&b0{8vkZ!)u))FeppkR6^f$~q*A@qx1hi~+@kvxXykTqhO&ofE-r%RcPaT(NKRcui zM29Q&BqE@rXjSM|`G+}X5H`fan9Zm)xMo5J=t$ckLI)x@wN)VcXcvytqe{h6rqpZG z4EiW1*@Z_61urVjwq~}@8%kG$6>j-Tp+U!i;VoL&Y#M4E2n{iC%3SIo@V8#okiV|5 z0l~S#4XrviIVBZVG%KG$%t%L?1c}(!5DpZh5i#2ax-Z&YhZSHj!}f3N18i(R-!axN z@GU(ItP|g7A_v355oYlhifB}INE9HLjp zGrxsD)Wl{Om7U1kwFT=zmM|APOQg&oFPKO&W0+hSn%mvDDP^?HYQ?ggI4AWPtJp}- zskoseKiapy=epA4#K-H7%=skU&MBb_Dy0=uyf(PMOD|L;!oo0cUD7_NEoCF!gIboW ztzlYRA%#*bL03k@wyI%C*6o-urIz#2u(FK0LC&h(?Yh3|BBo&?M7`Q@EdVW9H_4{t zQ`D^nbfJ|3-b8EN%jUW%xeF?S1ntt{5rO_YZraOG8Fa$LF2bS_brb$Jb{cpV`NLci zq|f8p9;0)oEA_*zBkQo)QqfzaV(=H#FcFqzw$TKbO{N1@vUN@c`SBWjpd}58d5nmI z%lx)X6=oAA?vUCzJtTHRNcaUK!5du+2U7;hEgnt!i5Eg2<{d06XJh;~3w^Fk(Fkh+ z5ENZ%X7B3T3DkL=?IcNxN!CEM5=BC=YeOhbJ%i(+4bj<>=_fJZK`-+dhk}fQI37mv zWK_0;!jq=E@=P7iT5srMOBr<>7>*~tWrw?dwChGRlv%{IbFv2HOzn3*F7(33=4EFd zgil_6`PChd@vKjG|1muJ-@@lk&=0L1!pa=u|F|3S3;dM4{+&`Q!`S{Q1v+m)jreXa2We+WzWHHjaNyh*_1- zQKkWHI7!o_ffHAQ;v|Wg()5sThPYd%*^2n7j?uMVs@Q5ynBjcX);A*Lnfno#=he|z zVej>?RLUgk;C7~OgsaR%4#~CnC4fIj5w9uXPB zSgACq&tza1#vqycdIJ54>LMSn+t31-2~OJ%n5Bi4(}|Xmt$}EZxB@tlh1d?N!G%Ks zD##isFVs6-j5U`{czve*@IrV89e#+sL)XIi-0QB+vy4VN*%n6VVV~%L|Fg0n7sk{3 z?F48TqR1L7U!0|xZ5f9p)lO+Dr|pb6coCL@6b%s~m;LC;M0xon=8hs%YvyQ(f5D45 zFxMyY5JBl4Q;8nNW;ur*XL6VHze>NV%0_KB5^T z>|jz2B3@jd6xE8RfTNtZ*ICIKBO?-73HiBMuVnLt8~bPyvZN8jTsx5| zaIJh5hth*y|E1?jTu=MCVr))Y!q9=$gP^r&4%(DCgl`*D-gT&g7rtwqX|lT38w%Nzfta4k zvXy6c{9Bf@OLGA}$qC^|RMIds+6Us8zaUav-+tn8Zma2IJbJT&O=Us@-dG1*Z1Lhm zS+i5`JV=h&n94uvmT_rt1B1c=saNh#0a0agw3>X&zo6TB6u1gY65nT=pXQ>|R|+-W+9IAzj za5RIF?wmLM475K+u9Y>l7eBIS%|yud zD8T$pF)nyc4SKms3uVwRg_vQ{5!MS5xN4P_l7Vvu21+}#kb$Wh+XYOAlC9)Gdhkdb zOUCFe+E$v@4_ih>YJyuGFkP;u2&TJPet4c49yp)N;H7nW9Npd0o06}=tNKYG; zP+3HTN^@envNTE@CQSFQ<*?GIJEd!8kzH{51r-`&P-Tc}+SD$shjsN-l zXAcD9G%r7>Aj_C~^#WuGg;kD_ntaW@fmdd9u8b2L5JSlKO>Z;jN6fbL%;TWsGP$da zd;Oau-=&L!f4$S+pyNW@h>Kg2&q5Wge=Jyq}6 zM?|Z@MB6EY3d5-3Zd&@Ss{-cT;pljvhVbw?#$?KBEU#f*MY3;6E=ZAQGpWofD%kerpP!cZrev+Q+3Ci zxlT4B*Ju)PP>HH!qN;vuW1%R^E~(sYUh=##dNXNJV^@v{#b1=6TG;4Rj8-zjbb{Kk zSR@DV|QLPNPio-?DG>zH`@ERJy63Ja<4EBX#sT34JkAEaZc&|k+R!)7If5r6@? ztSDg;OArM)MS2gWVPmNdH0)%=GwDF6oW{~HEMS?e|HdJo)@Gx3UzpQ1nZKKGb2=g! z-(5=`NZOj_%#lc#Bh?7HxXvOWF6y3&>49A4G_Dt=9M4$58meR@mCzOZl7y-Bh!Fj= zWs$7)h60&d&X($u5zR1(a0R(^>Aa41#4eeg*U{D+`_el4URB6@yOJ~(#Y`^$LN&5ZMgvC7Q>i6dN)I zukQ;Gafr&uT&Gd{J!Q>rBFC)kLVejNo3GRZVn=o(H3{R5e4pw`lu?;u3ITB*XmJk( zAmGEmr+vm-wB?$R!q;Hh@&n{qHFM=yD!7KRf=R{T7+&5EXKgIF$;3H0ABdhdCPln3 z8u!~96zl+Zz7h;wQoM8`VG$zV%BXPQbrIa83PsCX$YgPg2F=`6v5M*QhI5jLv5}7S zaX9M37H3VMV@Q%=EEiMZc2vaIzFd@hH^aAXCI=)B+fv?D(Wc&1Qlz-BU$qy347Bw` zQ*8+Tt)eQrvH~YXza|n1Gp|IQ1<;tDX#PepQ$D zl_69z&g-Hd(zu3RH0N#JY^+yI1^5UHQ`Kw}5~O80);LQyW=bMPuJ&#_W(rdhRp$Uo z5&;?~5hA#h+y!0Ekb0wyJa3YB#bGzP>x+TL;H9%nzH*V|F_#P>v#^7{s~pⅆ)~- zsZHD%CAEn_aKf`0_*S=B%mQ864vozc?pVFBxg=#pZ)q#HGORg9-pw2d66jBjrKK$^ z(AF;z2nqe{O!3pT-Vg;d=9O+CeGTwCj}Ito#$>HKj!o1vOIaz2xLcQmz4NnA4Zpq-*hYv%(HXh%!#3ot=>R2mSB z2;c&Ct++qi0<&y*IlmQ^8m-UQ83j1*j)fLoEc@3;iHV14JC6O%Fn|WWEGe1HORnFa|>A8ZY{}6i`N)Ov8L9`E*b-*9fHUxZRa30u2q%tZRQRBFrGZ?ZRNH zj=O;aJx7!1UGzZYGu4k2#PpRZ9eK)>cZWq@iP7qpu(Npf*{ zDn&^%2$+kpiPhOJbVEIpu*L$?8b;f5p;>i6*2-xA8!%XNmaJcvW^#A4c$?*#(ZlLX z=EEsi)#JmGb=zyk?2e_<+`ZM9smYuGKs=Y&0MAy6L^JcIf`o!e!W0hQZWyoQ z{U)`L1*-^$+^b?MD%HagfP=AOunqlAWP9_p=30h8u83LO(gO z2%6TR1jkmPf~2`X@-q??T-10=rWdK?p8KzI<#PR-zw?juqX+-xZ_9Hp{fqymf9ruz zX0qMqYF+2%n-Vg7k?R#f8BT--%mpamQHdG!MJB-MOU~Uk*XOxPF!EDxb2GoZ*#GSO zF%LcvZeObYSO0DmQ?Ru|Vv6xpJFb}k7GM*q3tBBbFH(LWh{(oOUNq+(p?5kxnlU&G zSb`F;9uRMa9BSm!9Il$h9G8>hq~E5Ng;Kv_Xmmy!7+o(2kfFu87|{kYISeK6txmyH zq|zltOuztWAOjZ7>25$y7kOf)U55y%rwuD7UD%6ePN6Ei_k`Ii2+-&xtv^I)^GJta zV*S+o6QRjD-Mmg`)2TVH;*$2ZZ@x99&ThI?%9|87li3$i_o<+oh@HR7hw9FOmew|l zC`gIph(JoAb{2>$q8=@3W0_A~Gli1W$btBln;^VyS(KGi-rq29FuSPL%eu9aYnY0) znM0trJ)d{onJe`oitg}Alt{0>GN<)^*4MaT) z37!I^G)z7Ew#o)>BVkyj8)SwbbA25-kg@~k1Z!j2JZYrq*k3wg({_5@0Hi`CZt^Xt zO<(2MvRX9Ku*WnPr@m@PK$0FTU$y0ER|f3P=t1fA2arhV>x~l_251s7v$2G2qbJsz zD9&7YnUyh@ZJ2N%byvBV2Kg(*w;5{&dcsgn`R-e{8j>KJ5i>Y@G}c_;Knh9KShj0O zIpqDL{dTQLKVChnzOB{Zsnhl#ow^6ght(t|V z7D|79kU}V^#*+DnOh-$3lls)GyUng;YWxqi^Qx!PJfRw+zNIG1cAz?h2bDeOwrK3Q z7Oq!!Eh7x9t%R<(TxbbNB-SotY6PloBwY+jpwl?YoFqTf0A#H<1f27{aEmczbF3n` zW$6GU+@ZBsKDr+Mjc4??_WtBIwDW)T{SUluyH9+Lp9FBwmutJ+e{+BS^JRe?xJ>5H zIOl%io(QH7kE{PnD)n5Z`(JYZ;_n_vpmAsXY5iRP{WB2b-8WC>pN~CSl7{Ol(j!3{1(SpQ`z?GD-{rkNi3x5p`&xyaQxd#gzz@A$mJCGTHgb{-Bsw zpJ$Q9GQC?YJQQf&qBFXn6$8>3VAw)<0hj?^9eP36=vnD9OVAS%3J_p4Qr@WbW6Q!E z&A9CoSdCplkR?6Pm3^cN*C3Qv{|_}bW0>)m2wvP zbF;=IjmD5w)j?_K-(6X!_LiHi*9ABQsH#<^ezmEz!&K69dYVR8q_LI-Q=O;v6l#f5 za6s!K1#Q=t@+%Sw1ZYe`?nU$)^_Yhf39HeeW&CU{*g9`Wa)%K1mS@i`gpCFBb{%gQ z#yKLw=&Q^I8DhX@(_UY$xIs5R(0i}{qmRTFd@0h3IXkVZUj4IQy{1jkjo$Zu=eP7j z!DXN0dgI}*b1PEAV$TKw#*W^{4VNdL9zUfYNe0E`TQ?rKbLYVadGtV)AyF5Q>g$9| zS(&O?#d~;p3Mx?l5vMVY?kGA2`OOmP>#fcU9w{+ zRuoOU%#uwkMYcq);rlWX6p|}MSDGt`+EbSaK)&0^dS)WYW@M#D(P2)QO9XF3gFB@R z5U(9Fm(Q~#R$$8EW>iHg7D%>`_Z%gR^f3;ZRtvhBjgC6z9lEhyRhgFBB|?{u2n#0w z0GrTikyEh1f>Sb+xYP?XO4kmJmzpYO#-(LT!ymIC>rw%tv9Pp}t%6t9HQcvOq&U_l z8+?%gw8^j&$#D?=T$S)tJjM1PH1cDHrBWG76O4|B|LaA-(nw%NP;5v%%dF9jq7IaJ zV8d`%AYs3%Y#2_C&~u3fH7zNd)m^WQQ13fo#DnUMH20J0w7mtfrrpa?)32xl%XZU} z_EC{)Ptye4r4YTOB%sIEB0=;~hsBcvZ#BXE*$|n<)zVCAo;@1L%x50Rc+xrb;xgIWGcA4oA%F6(WSWa5`YWD08JcHyV7Lm5?FXj)n|16UnyfOeAClAhszN z;?4DJ`8O(_AY6b<$MXO|1Y7XJoT5{8Ew< zg$(Ay){PiJ2jY;!mA$INRvtRUYjkoi2@fb-GtkI~&j2C{;|oc*JS6K1axZlpkD5iJ z{)2_o39z|R^h!wsk^0uW_- z743ia%n}HhIH8-#%@m`;rMAnANtp-*w~YcLV=AcXKnsBkm=}#cXo|#bRIM`6_5X$W zzP^PuU4~I@m<(T&sim!(stP=b6k1FhqjFa6DjXfKY{v=AXj!Fn&1$;bcx8!ll~tgS zlOnu|vM2kYF_jJWRg)8OTs6C=Y8I?vs~FDGRplox;Id&vH5D=Jrr1P<7k4DU=qRkQ zqoirHRzpOCC%|Vae>V#P)H9+j#31C-a}#VBonx-x2?Zq{gzVtNdufM-c$@Q7>2{Ch} zb-vuOGha_@lXs$P^rXZLbx|kPsNE44(WnYl$aTB&*f1@buLna2U03DaMFUdwqfn@U z$%t1;8r@^M3df&nnpMG@%Sr5;VM#uC8>mIOs!JqfXat zWU~z6THy!5;l#*#$lsBNK}{gfx0$XUAXg?I$IMqDkXM6Xjcjbz?M9?8uh(=0A)WI> zRoDd3S!Qm9xxNwhcpDhuWI-qJ8lnmv6#&O0HeBtHy3H&xIigc2tT>K{Cy+RxLBO|(fYWFku(_DMMA;`LPShSEfHVX=A*W~>t+t+2vxw)|}DCy?T4A~J!@VwPLl@JT0yc8}&t(VJDniWvU8gw;Zd zy5T0el86#F(*_uLV4S;If;yG@^UNUD103S_m8#~r!XeCDz zp;u53W+E<1t4*Hpu&Q>LOEiFRps2&tTG7KfJ#Z*u?s(mlDR8yTO1-xk%f5fKy-?~^ z2LJr9h0LgkE-Ox)IJSnvUXry3qAZlsKkm6<57}4Bm`tFd=7zFKl+&|DOB>1D+46(e zdP52}6mYfxfM+h-ac(iT(Ft}f@J$cVhZ*Tdx$HvwVQ!<_FTCeI!-LaSAAXRc**Ca; zh3lo8KPiGp|LZStQ}KgGJbXLgCit+R((>w0-`T3*Z+`eZ*HR`Y3tQ+XfAlGCr-|GM zANeESWt*T(wjW4yf3rRU)P&w%hw@tr$AyiX(^}?{<3h#4*F8uzI0IksB^P zbL$zs#RpIEg=pS;LBo#E3y-Xbxny^>L_bV&9X}K>9Yi7I)M8Q9Th8g(up9rz2?`jm zfQSGy8P;8yEIk=dS7qw^FJ`RbYKVJDFBYJ!%OoQrg+k=pQ%U$4V$p-M0{zKG8n-#j zn3-4V6B;p9MTZjYoGG;vW1Wcq34LYVTJaDX;WrBto}m?n@(j%~&BoDJ7--ZLQ=wzK zPU_kXh~zNRQz#PB+Aat;1nf8=LPvDmg+f80si_{Uxh4yHt3S*JD4x}ahS8#g(94o# z+OV)xX<%GW#&_%N*UNmSjwpdLn#QJ<)c_=0SV&rz_&{W5x5_vmSa*o8a+ud%!Za~X zY1m@`O3W}fN(1TH`oOi`knVdwWP0&w)z%3_5Etr+^m~x1!61bKS!^j6ct`TqtKp5K zAO9i`q|Ev=;=r_*A=p)|tfR7`I;X`+C4FLyIsmwqV*`UrS6rp5Od-(Bh1iFbFoKxM z?^WbNx_YPvb`MPtrLmb?a3QOXjG2S#(tGe%&Ss^G4gd$(&~Q)S_aOq%ot4*&_{Ame z>t#8rQJxJ!oUhfR6RPks+jldoZ15&kSJ=XF;BTlg*l?)J)I#G;hlo5Y^4XLRnK?9> z)X&0A8eIEbqB#}{bUHI$dK|igX4j`NETLnZECPdp7t+IP8H>RhAUsrI8Ee2M1wAc{^A&9YAo-fUG(@Eu$43DHHij0^Kr<`fs9Z|tw zDUMj>VkYrpY4J;X)iNJ#HxI!f1T&PELKZLr{ffCs<0PjBW}j*CYppkAqAU|mwiq^{ zieCiyQv9;i0CcJAGB{yQ{LsLcwA>4*^~KL@_qgM7=}&(CNg=1bZ}i0r`kB7im6s%( zuKrmWeI+4m1!XK~_VWewk<8Sg?MTmjpjnz=!3b@yB`5Rg4l|q49n{982eJCbscqn> z@bi;%yYC=Pb5yCp^oU(CkWNFEOf*B8=qHQJ_Tf9D(RWj7082AS}2*WQ-+KM zw2HuOn2lNRYDQ=fii4cnP1Qjc{F9lo71FVBR&cKN^@mXn7Q)=8Sz<02cb~;jlhhlcROxp$}KKU!*%Pf&kG6El8?l(usw0viPxhHObXUbKz?2y(8w7v9hwG;D2d&Kq@EeZ z#(KsURUI_4+24$*WR3vkenxUIQyLuzPXr2S-7;QZQA?Z+s3m+aTZm91CIR+|FgIU= z8(@+{=1(!E6~?8PS@{`@+^qG6DBK`Tx&_;M_7_^wE#@Ryj7kc2a5NnKx$YVdrLkOJ z#I;`iuEXsU4}|Y=_b-20|Kcs~KJw6)zkB=D-i-$YhT`%yt{)pjugUN3`2+nVw_l%r zm)pKTSYOtkKopdH9Pv>#3vpgtpqKXeNL5Qm3_4LT3lKI_&Z#}gbgo^dNvP0%F65?T ze8WM{+=$F`9uzZ<*34B)$aMwf+}62)7JgWn`z~iqI^+AW0Wf;?niO$1p?}m_lXCMC zUN-@3MjC*5JO)NMkQaXK*Ap9QOJxfF7$yzIAhwWipg^+LZt!)5YtsNVsRz{t?nX|qY$!3Y z`Gz>@&o|`vDh;eZ7xk`ck$#)`dtFn6_0J^@2TnCq*j(_~0 zpFZ)*wL4oM(La7t-$@~W%MavlSUtI#SIYZxk@ns|KYT(IMYw$Lac&-x#+~m5Pn>Y~ z#Yev`HGr>6L-GD29~O%HWB0hZSk(i*dFhY-v$vf0^9+}YD4>7%;G@^xCC4Kv2qWmTwPhURF-KV}c|IvHwhyUbn&wnx>y#CMr z(NJnY=H-RGcU^vdNxtqquCIUV+sQPLf9>X>Y-oLX`d|O`zr~|h?pJz!wpR~jmg@CE zTHiuYn^8*ig?I@w&^<{RoI5VBbN$ifs+RVD{X739Ki-js zIJthGtDKNWxZZt{o3G#RKYC|;`4iuo@IyTqN_8LG;fPs4b0X=S2M9}x))D-hjsPSe zf|1L(LHe1_Wz$3Xm1>rhA8B2#%{n=+s8{nGXci3JPVh~`y$S)XyC;|V6U%39QIn4% zLZ-#`dh~edG=alscLJUWVFXBt6g|dKv%!#A>?Jm|NP=XNn271gvGP)8Hr_rj9vopm zIaGe-NV@)1_$CA;jhEW<)kV^mbKW5;ZO5b?d7MIJ1&j!RH{~ zT_^~2v)E#~xL~XYInuXa86;%v{+NW1t_ljGa7{BOF3-3Rb*(pyBZ+ml5Y8ms(EW0e zfO#(IhP%VQqNRDBZcsJgBy4kiPE7raA1Iy@U)bmRnD`xe9C7op7r6P#)kI9{EC1)O z{N|KA3n@dLf5UU`Tg#7*l<1qkXNp8VL)io+L&^2fix%{TPZ+^W*V z*Tu_!`fG>s?hpSYH#fM7me*7A;MyDYQ_H#30eJ=qN(;B;$j^F1Ut^9J{JHq~z-@Seb4uCY>-&Q^zH6lYL#g84_*g~)(h(WSrE zrzyBb;gK=Qg$uS2p`GCHYcaZ;4BGmci!_Vr6RdCD?pXd7M@BPQzewOQp)d}SR7cmp{vp7p9W|C#H zDM!=8st2?3mf|`4u-XLy$5*Ou7sX?gNhNdHT;rrFGRsY-YRq=k?5MtYcaVtsfcTvH zP6JYLSHS+1=8z@#M9Rc~5*)Z&44br^t}_Y*$G8z7GSs@kX({DM%PyX?D441j2ENlA zYd+C~pH0aBv~bS*s{x&czr;YpT#`Skm1h9WRr4e^IIG$AB@vM`^FJkraY&;|bNOU0 z?5d87%3HRl##xd4|Lnc{lVs<8-baz(rAZb^5g^42Kn`)?eOBG)ewJ5Y7r=5CO8~nI zfaET?Jm>W3ox#pd59fl#ED1ukV^!qHN>%K{QCW_ipJKTZM-|1YM3suGoT#jeC0R06 zE-5Od;-peZIe$bx&-6Rt`1fXxzJG*n{?f3m&pYQX45iTWu4PZe@?%B!_f9ON&FJDYYgFQ!JHuu0Lk|(r&Lv-(?%_m zigDP2{mk~~hP%PtT~yw;@7dPt;-opN6*JXpc%Xat*Lrh3;tk$-ivF6ry)I_EefSl8 zXlNt8{&fPOsQNnN1!b>^x_`gVq%x1!M+c#b8G^#Gc1Ry4sMf5|!M7zD6`AXq zivd8(oAq)W*qin8>s{^IH|yn+oKt@}*SOm`g1~x_XEtk}WfEyM?;sdu^7kEywVtTF&v!Mo zu0i+w$bO>{Ag(6noQ&Z`!fWPqfjpbwtYN91g)7lT+L|y8bF5|+L<`zidc@#l{SN52HPt%(qM2W=xuj! z2g7(;h`xVP2pK^S5|%eO2PY-;L(=xlq88SW@qZH-yOG@zEiVd&@el>qnn9fOmNmn4 zd44$GZ$dFmzWT&Yg>GD49e`%1<6ZhE0{~hA)RPp81nyB1ruLX1b1CCr8l#MQb)xQk>%TxBoD_0oH$bk3PA_}L852=K63kc|$uZ|xrK-JEhhzEXQNJafs2DZ?=g_NO4PKt1G}sXqm@1V$=` z+dquye<452d`Kjc0 zXFkPUoV5Zp#@phf!95r);=^MX;(LXON?J&`ZDZ7H_x%zRISK(f;HeD6q`{yM2k&$S zJs2P#OHLe|9bTM)jLgoG#jU)= z!S-uzbBt)|Oxj;vfQ4wi(iJY780lc+h23Uh{<21HyA3WxJ?ZIntAPY)3Z#8J6POX& z2H-GgU}Q69ut3OQ3MfHL+j_CLEcb!yl0b`@c@BrLSl)?^@Z(>t(W+=NU5*t5s*CKC;* zV3c4r^D&SjZ%X@i66f$ygdNt2f$L}lB&!G$OnM12m3;~2Hw@1-`4|=HKM`eV?>n2yw!#6rkeLAOG#&|7D&}02_Sq!_Qx+%#c|OnIXvaj17P! zW2fA;tmYP;C3B z^I=nK*_-GLR0d(Mxxss-4DKWBF_SP#M%I@yy@Zxb93`lJX|#XY+?+Hzmuyb9gVkxI z2%uFDp5ZY_}Vt2of3k5fXr5nj)1myOV@To==z? zj({Ia>l`L+ffbQkMsI-JnUG=H;ut~CYa4>Ar4`zTF-6g-eqa5WrM`EfVj_SuUjNi5 zVxj9c7G5xE>v|{3UDU9ig0E2K$97fCs zc0Nstv7a-!(gJiE3W%k(+R5%7TTpNYu_#K$mJ91nc1(xHSu!IM(6h&AHWMn@4}aA? zb^$v3gN+NNIz({1Hfzm>Xu;U>ez|cWlXi}~qks=VQP<1-6RSdZs@-a7@1X=aj+#M$edp#R2yVus zDIAJ44_R>^4Q;d{AcHQMEsp)!B=X?7M`pAm3SgVWkZHoR5jR2qp5~qjuP0zIx8Q{@ytnf7<`>4(ldb9CF(qfmk0mHi zu}|+AE(k+O@6-b^xxiAGc3ZIK)$aY9<*T48^%t851onALW!~%FH)+z7<;I3W*{uo^ z0x_(|gi)`DpQSSzxqK`o-iz#+2#vxP(kYn+iwFn#NS6)EE!@Kiy`_7AOd(J(LifZF zHV%*xDr;Tu#8fl*=Z^N7Oq5r!Zp#Ku1`E_)LsoDVsvcXbaMNZV`p;%UZ{E2XrZ>Tz zLL|cEEQJCxa}EndL3F3^Z8#my3#A>j<~t4xhifu|T2%E8$bVq6fsjmOQn6PeEHs%E zW5za_vf{8rj7`~uK28SC%1T!fQ4$H^lyjz`>{;>8h6#$5Y?p8W#$!&O9sh{Xc-#cs z5&T9V@HGK3%6pZ&v0p_|?aT!@WRk5c+N;Hzr-kR7XAqQkPCK6)=4%ied0()EH%kT#z|={BNCV))P+TIo`JxHhSKdCtfshiD>hW@cq5S;kRZ9U?*NbckLYzw{VSpW^-< zA>sbFzb=?+HDn#FA1hg>49Z{@A}zQtI9+^qhT5fOUamKW)F% zk*k;K`okPEoD!fCTiZPlJD(|NnU0r^-?bMxecQX{yXp!#b$JzT0Fi5IN5^u(dTdK& z_yPBDnr?%YS25pQ>BJ8d%V-}k)xCgQ8k5=QwiEhmIrsJL{FobR3_h&0h;( zR!+?hF!mD_`7{BDj9Hrg#DWEoU+Yd@X`c!S>uX|}(}#pEv$hO;mu2R8Ib;O61q`~D z>i~9-Y9qOZi%zLc$fy8RMGt4eIDP8zRf8m*i4b7wl^)KZ z5O+YU#YO~AG!ISNbevM{1hK6aA+Y^yj8@((MB3d;%ODF?#HJF;d54ff=ij*u8|W-r zY%5&}{AAEYEP(mqV+gL1y85B$vNZf02aB^v)N$tC`xOdIFI0nU-&mZN>a;z?8`W7c zzteSQ#H_S(202=7ws#Ov)BcfI0Z}F=`5fm zG(A*E)d1u3>v#Ht6*Kg@gM!ZUCR1UQ6frHJ;bG-KGa1HM4UW%R)kH$TvctXz?BaNN zSX|@gilSDRtEt7)8b@V*o|HA)p*_QRE2*OJfHt zgu6KNqM*OEv^u&*y_fIUX3bqSA)K8w3CN!Mhs3BOu345=V7Rk!g<&*zCtbw=ZY@f2 z&BdY=kepVv$3wBDIg4G$<){dmJ^>L0!wuyj8qkz@(3P1*!4$SS@F^2>E$(7T*TbY* z?p7kOHs1(q5{y&sG9eK>v`qMu^_EFkvQFi+$7y2j^px<)&R~Q()gV!=Zn|=GX9#6n zS8m@S-o43Sca-@FT_F%Skkzq#2xM|3k!nIO18WXXQrQt$V}{4+DB_qx3dGb8qsN#y z#-bE)X+UbmJWxC%Z4{A)6+toAn`6Kq}9XzR^!0pEV zYrp+pT(z%xeC1<2sBLX;pLxh$nq8vn59em1W$>lcncKq1XSW1Lu(3f+VNVo2pBpo0 zxOyl7Xrfioq!S!m+rGm;x8QZIgXr^Y9>gTfE^eN(agc!JlXd80+KGVjLL6%7!Q2V} zSXeX)K^*pl1c(<>^2B~m7?Yzsi;IH@g{la7^q2Ph-fGF@nMeq-b5Woy2@2MAO zX_%KSPd6o_Y7KemQ?4LB9RqPGyT88!OXhEbnN_{sW!Mj(9)03?saEnd%$nP1+E}Dk zbEzie8aAI944L}tBs0*hYGG|gJPXGOX;V>@fD!#{oS+t#Vrcne^tzWndwxd*6MJ#wK^*dLk|qWg<#J!wDQeC9-j0 zyD3*}LQ0?ycaD2IuIQz%!bPNvwL2bQ1UxM~x}r1ukK%5YDMBC=-cCumPGO2}wF%%9 zXhoKMzI&iVj4{nf!EC`&rCVaEmrs^X8Z}<8g(BXpKJ`^kUJW#FqaP3mL-?xu>>=2( z3WmCkRs(zwbl1tS$L1YEgXm^30xS}v&gVf(3EB^F1(n?>Uly||ejhoAB+uKKzD7jC z5m8t=D>K@hKE!J%kRsQfX(l*;8kBY`O&e2ogWk~e4Uh^N>Km+dDYTEEg{$<8P5n=) zenf0Q<&C~mGKLwz3Oa#TGicgb<*v;&Bhkod+OyrUt#VF5-jujroo}mgW*`eTq~=)d zFmubBOb1gPb-c4}k{Y>yI@7N`^nlKAMEA=?^}Q~dypJF-NS1YmHJ9QeiJSInFQ&5@ z$oL3LT6o0Bs9qrxLs*t(P~6jN1?1iG(X;J++a+}E29T;oDvEmOz~(0eswkCL-TYT6 zYeTHLzi|z4X*9R!5-Rb^&Y&=2PT;hpmEw?+ z)@GZw59uI?4wDhITBk@cY>qPtT1=!7@6#vBvT~B30s}vQ*J*9x_M9+#0S1`e8-pwX z94wTCMq*UEM|&)vkR4N3T_7EjA*91fSHk#^io|f4xVcz(){oGS2SOQ77zeBfPb?}3 zPA?edxT_iQ(T7DT^c5k07PqsJDz6KRY&c8nV3*n`RhaA$lgo!<$Rix1dkMLNTLkfQ z%Vu30+@}~w+U_zJ^o)q>we9*cBCKubfp1@KEFd-$`*fiCi=<&W&B_$uW2nXgolu;n zhb{CbQ{-JHI8W1#b;H=Zb@icf3iQi3R)Xao(q~1>cD-ciNvs!{7}c#sX3=Pgp68D{ zxCtgio2Ogafs##(ssOwo#1g4^k1!1w2!L=!VW7|j!%6I?!9cMyw_REse%B*1J`SB86`b6*roW@56dhy%29&9m_R_q zsh_F73~?aR0>x~G65k|kDcutBSFCM^2Z1e^*cS4YhFs%qqJiG`a=U{80Rf)Nt$BdE5=pr;v! zp?yc10MB~DDD-AYUnt@O^Kok(j!VK^boQCc%Za$qh$mz)MCSqXu8a`IR3md>tuq{# zUyrpUw5}Q*!RFB2N|%E06tZR`m@DyId=mWr;We;D{`jm$k7BA$RTYsDylC|PZ&l?& zd{aeTX+72_|JAR2kymfOS#{lcEmAL`jn3_STGU#vzlVoo?joj~Jgs7)JRCR8={?N- zH~v<|Rk9O$vR6fDkPPHq&#uOEkAJ$SdPbCF%PUVm@<+#z68Ec%C zZ*vNa6A8yNSlL%zoI&iKkD1?O@{SA?EEC2JgNFuEUSQog&MjY|;1yJN2YkjXieu3d zzaKVhPw0A;{m0EL9__{}oJnhCZNI2$*)r(1=;k(;aN@<4rsh!SJ-{Ue;dutZ0689Q zT!v~>NbzFE=+khPXEPLy;Nn^~Y=;(b=EfYBu-aK7sG;F?l_Bm7xUGBAPGp{Dk6zhL z+mbV?D}X;JqnMWmwu5%|;m7yUi)xK7P;a~i;aj>Mjg71mO%L6-ZCl|=~VO3K~yla zMY(BQLOV}yyady2j21)90v2Pmh9E^NT?#CdK6oOiI2J}Lx;FOChGrHI3Gy-2o|WpC zMHE>hB8&6__uu(hAs`>({?X%I4daE6obYwt)fnEZ?5pOC;zxTz*1$}sH zWvO_1>A_7NxqZ|=@%a6>|M>6obYimvBH99KtQdn272~C+O2Sf>>P{(EKn)37eQW;o zZb`&RO@*}2PKF#p<}Pz~aGP49TV*lA0#Jyw6bA(4+h#MhHurYmxT{?4AQAjy26#uX1r#J_pX z2=pyR9i2lNqE+Pg79AO-U$gG=P1bCV25-CI|IE((c-kAlV!a{fAk-R0N%WWes$Yz&7u%f+!D>LO7nT+hsF zi)G~wf!O|SH*(dUL`xhJm8}jmn@~vh=k%L+VraTeENEA&J7r3AN&5zy-@&*uG-{4z z6}VQsGU$uz!)zLTpnufqPu*Glisuf1`CO=+ulLD0&0qutMV6$LYo@x&QtBmJ? zVFc(%_W%~Dd5FMy0;R?{r^z+S+n3TC3}kibAPG||7( zmC%w;0)WW|Kw(oKq6k8W$ysXoa8%LbsYm%BCHc7Gf`{(>&R_eQS6(w*z*Wrf{LWWD zQ4RB*zp?wsc^d}2f;-Gj z3BoDusf$>1G}Ot+izvNNp=jHUbgW~bGu&6#nUNz%=b|4I+xFXs64lU?&WjPZt@LZzSa@;|5Yb?uE*wc;uYntN7k1GAme!oQq-EUdSavTM5W{Luu ze;`oK0I8A_0>k$aGYh6I2t?;NdTFu)fHVdiahF11!_7kqgAPKkr2#CmUcJSFML_A2 z#F#B*&5S8qb*CzfimyP|Wag}!je)*-VK0c87fwTT%*JB<5>UmAtH4OY;$|7=vfW)r z%&;~;3iQ4`?;`F6zrc~?pX{|ioD@waOrlmP1i=96xIu3l7h2($ z>qOaNIp<3CXriPp_sy7RrRv#lg=_^H7vB{*xh>C&BtdKw`=MXm^>GIk2+HZpWcwDrwv}z0 zl~Td4;nq^w0sfutn2;{+dY!DtcXv0Qftbb|JZq(mN4uCEKowcmUBRYbBU@j=SgW7h zvsh4FQF8&WR_GKGF*5?=BXo%v1wAsaq!ZpsUJpBvedVrE(s^0yVewd?`6m6178-r) zZKNyIqY8Kj&&}@;H93o958fplTt~SCP^WKyPXA`AGK!`%>qQ&~)ZILF3!p<{jL)Ob zCze1WfhmNy07fAYi%X9pT1VlHed~=N8i~drC#keCY8oJw#gg-i>Tu$FWfYX6!oZtV z!^jlHhC=3*E+u423HA#b25#Xkpvip}0!;>^=p`=-Z{g_kbNmsv7evU%?dn@M-`M`q zJJWNaX8OOtT|EPPyYxrDblJYTviY6g_%-|TgFK0L@5=ks4!F(z2i$j5!<$Pghub4x z+}E%(w`w@)Ygg{OapOH|Hr#jO3O$ka-G<*v5jL27GqOo>?WoF{l)N!}ga`)}Iw#(>c-~*((mQYn4}p*XQ(59GsYl|r_l1z z+@$^==eeK7Gt&8GgCs|9F2=D)#T@=xy?G8(WQx+fo~cXF05n0kc*Zb%bQ>lZ0MukJtP^3~X{9xe#6W3n;^p211j2Nh6D_2~e2Fs50B8=y6^fNft0Fl& zs#vO=mHYk?#@;pFViX%l=qY!vhc&*&{Nnz`ldnM#yJ%2Z8-5-pS-Jvqh=e-X639!e zpXfJGEKz@rh~blI4mOx^Y)ut=m?KS0A}*0+lY(4uRJ%B z-DfF;?m|v{POVRO;i}FYe+rbmXQKggTxA##)3Aj1>Z7WS0AB?oV)N-Fg+(hw2#j+n z9$XZWH)p8vBJ)R5H}2{{5gi4zfubZMk=qPs8fp)z`R5|!iXesNn8S0+SSHLiH49Xd z!~$07 z?Tz=m@_mh}zums)pH1zH?VtX>oM<({Zr|gvV&AY|fAWLxG)d&W3)_c4| zf&j$FHn^tbEw&ra*!wO6J&k@D7a77a9UJKh1@*2>mV;+R8lPDA5yEHZ&83M^a{+bob~a z&N2ZPmKSQ)K}~@#)nwE8XO3Q1_xHU6b1*#bw`*t z*S5uH{Zw;>)MOShKH`1j(JMxb@4brMKlX-eVBV^BIk%Jk`D1UW*KTizKjJyM|A%|O z{WHAy!4HrA_~$R))A(Rzx9oR)edFwtKN39X-pM~bdtq0V_&@sjXYBW=3V-S578Z^O9O zNgT;RcQIL5`qNsV6>_gTW)l>Y?kBw5sMCXRQOt(a2Dl(iIsjq!(pK&SU zYX*uIa#*IkaBF@7Q7EJZx*B)QOr6>8f9GiCL>G;oF_6(-5)ji+SQC8!@#`V5444VB zhG?P5nl8={%Ds-h*O(f^3Jv4_^b#}~1q-2B=slA&N!mAu3?!qS4hJIA>9pgQ7U*D4 z6$yQY&iXJSV|E%u`3LaWx?5>NJJSqpckU&5&GnLTTQS{uj;%y-#Mw;d5cYHkp`?JP z>2!ORg!_p5pK6RarokP0m^PW z+pCC$>WZDg$RQ441HxwHhYkjZNE>aSWG70%*1RE^DNGAcBlmX7i7KFogs}uiO~FYL z*QqK|8GUkC9{gDxnXTnohQqt_Y>Qwq;2z^_|6I7n@2>;gY{`Y^bd@Av!|h+zC*ngSbM%%BA!y+N{W?6Y9O26o52L0emQr3fgy(;3-)F zo~zZq15VLiv=adC%uL!jyDOkMWm?yLOSNWa_$vKgQu-Flq-o%Z$@{nmG7sQ3U7`Hj z-|Pg00{tdjbmXI^{L3s5B-XTr8Er>dxGc!O*%Pjr5=qIjouYX~>Ds*QlsqX*EN!ur zf=%}pvoXzjp87ed+U(w97LuZ7Q|eoG|7iuCq;Zr>biGYWM)8M!4V!4tVVl-xv$fZU zQ(@9f9vg510~t(fFgQ&}2xtYWO~lg~h%5*aG5bXqUx0r7(G<+@G(q1EsVxZ$NURKk zbo_G^7v7ZdYsk~#83xKyx$JnTwYSoxU@2LMk*J32K_Jj@*fY~jk>iMNiNt1Xq z9$kxU_H7^4o!bx${&H1)y#2uUp8w({y=%Wgnl^7Fs$hQpJ3Oc+aNMX}nkoYC-WYv| zM{yN@h5I&l#RgZ$JiPy%Rd`_52xv<+5^q3I3J>6coe>g{n6Rs^ox(F?e3u}rJ2M4&wZizFY6(3iJwjT8_9_EVV8I~hB9JvK4zpQ;3q&JRE`!x* z7%BCk=scRw%vq(~BcZVMZZ2trN*#wiz%-a+;rT>{=8Oq(*30TIz-=j&GA3mqMle6W zMu5Ye(Ovvt)_E1U&wz19u!-ghXbc9UFhytwK_5h=yC^{2!ZhU5?OIy5yoAi5m3>tl zgM4qcd(1pa6UxE}JcHu=VV^r(b13^Iz-xfNU*zN}%I6=h*2)YQ9A>*!`MYV*10u@P z4SG8RvurQM#58B5kXXFwkEYCD&KPHBs7zfy8u0*>4T2B~SoEPMOl-HyxMX1Pzr@Vy zLI(yA!(&h(NPx&>1?2H#I#bgL1aV-BWY3B2DtjIb%}A|uDTD-oN=Y1Ddybmogf5i` zvKY4x>l7@Rj5O5rkY5oBrrS_Vr^b>3;V? z+4mc}FY%bX^uR-!v*PxX_G$6>oBx^ciB%={fp=9Msbe+AxarFa_jve|e^kqxnZ>() zf>V%@rvasLY?<`k^0h;^SgUPO6@au^YJ;xm;NW=YRT@zpqbT4$V-r31T6E%UNM$_4 zXZc4RlMW{64D%*l+)ICuR$M+Ann8508nxGbQT49#lhV$5C(?NBY=$w)Q zhL|N18K#~hpiEOr2*m8;XbNjKzoTo@q4L@}vMn6f@xe?)nbBFGz=N!^XMt>(xnAA+ za=AlCs{=cxjdvbYXa8^*bW8tE@p+2&@yv$`E`!#S_8S18%mb}lLaXcLv!omsEj~Ci zeZtbYYa}V1TaCIk=-b{{ob$In%!fsiDBihd;o6iJ>js=Y(5BH=4e+$8wT3}@_t6CX zUB(T$;;BMr#t>{Rmd;y5DFJrQqLm}}>?0#X?rfuo!2_e6+U?ee#O+7BYNh1@%q?)7 zY&?bfo>roAYG#ElFr40jKSXd*83^AzR8%dMLG~k;ZrrG%Yk{91ypM;2FY{n;J3PJp^B1J*^u~L6xb#QAXa*J^ z{o#8|NdMWF-&d7V%$i(>xN*8|dJdq6ZXH$0*1;U%9n{kBI(IX48+g7`&Ed8Jar(OY z^O=P~LAf%&>qh2kBv5Cm>jHIT{_O1?cw+B#MCIg3u?c8;qjIFnfuaYZw|8T578JdL z$nRw*V8$g5A>}5*1nfl*f?f=pW>cdcw1Mc*_^p7mCXjI>DU@IJ8}A+E%n#{RDP z_3Bg$x$~#s*J?6oDrTqMy>-;F`x5BfPY>7IP%tZR-Y6kh!E-PfP{>lkL$e(4c{~=` z+k_r0MPB%?K_oMe!_c&$M3)KQ68V;?tJ_Ab{{$SVF{qxA8UVo$m0z#LYrzqZFkZiHkDug%uLGZ*BXD^dyiEzS{2 z0lg0J#H*&Vmfd+8?%!J_`P`mcf~(;ooSzyW*UacV_!CY`S->H2Ef}b-u4fR0Q#$1; ziNWs;$+ys22ql5Y^_Zd#^So@S#?0_D%L`P^o$GDWYIK%r5)bz8=%KH-ROhy`Jz;B} zu9CRQN}E2z1;%H)M#@u~!>6y)q`!i~9)m6!rX6~CR8XcF(D^8%i~`f?82%-AzFS46 zVh)gdw^bBh(qn{2W*eNYySYM-5^w)pIaQT4o!T72l{qC|&Z7C?s%e+GkQzO1jc^Hp zVs_0yGR)5EuZM%FrT<0?tz>9Q!v?}_VK_m2r&=-88e8@7;>g1AoRg>c2rwJh0Z~zD zUQD+S3L21&QG9TLeJ;%uq1Fup7~XKz_Ob|VW{VSh&P`;L+_WdqM5@^x!2>J*xabiy|#dRd# z1=b$!&MDn4ktUX*8k}AggyW5eD0)}A5+nn_W-Tass~{ZCFd#{hmZvy<=RjidJqr#$)0zx{jv zi07F5#x)Hma=X8BtY%4=Po$$^LQfi)w~$zp8@rReLYIhmI^!*EwY?+c)o^STlG5>Q z79;?*p{XC$zO7qL5S%f!H2%4M*9O)*!UPkU@NG_n&BP5rS8C5u(AUXiIDk-vSY@2D z%z&S8ZBvo5@(5W&%rsA{7Dss48dS9XvBNDhip0>SYlu=Ba|2$EWNc_f4z)95OhSy9sRJJJF+iMhf8AuBH2dLK`thfq6gjD2S*k%03H9+Uf-R*<2cW{hg zXoc~L0ai!2YV%yG@le)cGbWIJQ1YF!ALA*)GNf%~Ypq19XZ?y}x5XT*MFkbPT~Aq3 zifsf@lhiwt^d@hJ$g331zp-la_RP_4wsF3u;;7;RgCo7B1lX)&GkyYZSpAG4qiuBM zRx{cNQ_;o@O6C;{54yAvZDM$4VeZKwPGfPYfFM8V7n6=QretA8)7%f!VnRC$VG`yK z;1SEnr8moBhMGmdoC<(C_-P5*O%9SE#4s>vx6-B1UQ0zSWvCt(Nl0=w9G|7T1B)Dk z6m{(cN=Q{c?CoJm$i72{`lCO1b@6Wcq%sR`;rrhHz3IfhR`uk%$|-um?F-y*ZT#*( zwO{f0Z+`Xf@+e;*aLO0CU;8W%_LlRc#HKRlW~IbX>k_Y$yO?oUcFbbBPUhY4K!HjE zw6eVmg~o9sS;=mf(8T5C?lPFQzixt>Jh)7LOuu1|z+N$087D2nPBct2G);PQ7DTN{ z`o{FC`41o>MKW_DQSdEeX3@otwrsB0)$}tfss-m1qRMd8Y}ZLXW>C#>%@Hsw(9}6{ zevZl4ZA5JVd*~x+8g;NXwl6qz@FZo^lek1GKNaNW2o`X}@U;X+2!b)@p)I^Syp zdD?o8hg*R0b$ONKj+3_pIKQAj3Wo+~y~gG_YBskvo;ccXsDj!Q4|LRFoiH~SitLyg z-EbV<33cxqnDeQ$%0l50HRZ&3o2n!Mz`1Z5Y%-)@lyJ;4QPCyCM-UZ_N(R@7Ov=ru z5K>km*LeyF{48cB7u*~`7c%UdU_ot>BniVo1GU1H5c8rb*esGvAkRkKIzn5P5!6cg zfmG8UeGEgfCHwOH+jv;qPM~lfu^W@-iDaLe*sV9Uo^3pS8ghAZU;7*GvCzDK(f8LEr>(Z*TJ{6mm@qCCw@fR4cBx!Zm z3bDi?JGM5n>UrrvZd`;aLmtNsPHm{QPpzsW)Pe$}8Z#SQG7<`wwX0MG?aquUOTw-a zvD}y{6?Jlm8N5kaq~5d)$`&2sth~FtrHc8{PJ@C4q}ds-KX%9Dt(W9H!LEV#!OW7l zt8;OFT=auiCW*G{2+XOQSCN7q2P>|@U(;>wH_)*Xh(EJX4|m60c4q`TP=p5fbxKPn#u(gMFBK& z3cX~Kg^hAdvbY1xKcRoEE*1>X8u6>g!!uHVx6n<6hGu!NnIicZLX2_mTr_nk^pWZ2GC`?7N-v;cQVDzUr>f;abhs{6t|liIIxy-y?j?UtOwxJ zdMV!dquGqn4J$8Qndu9#jqUe@Ptd2Xpq!;|>9IO_!tkDGD+GL!FC(e*HCPD(c3dWm z4xod$v}^}V*114`=eJL*KyX`aKvm6beodpFl)M$xddN1}`I1 zNYG0|G^IDbyqA#s(Od39!7OxeBvnw}pYQ2oJ=)a=&6i7!UMB8`V2AjV^;$24NxRz! zC^ab}joXKK48n4&S>IeXXhT;;uQokfZw4dE4}X;7hG5&H5i_Gk0@7qm1_-6mtcVyH z^MWaW{^cSIy#PL;$ta#tG?i1s0h0cdU_N1Jr?w8i4I=Ig=M#%b1gNI`;JpFQw$h~l zz9ej1PGaCPf~8g@ALR8F6Sl{-l&`sM%a_%^&-I7zny^MY z1-GJ+|>0dqasbMyI5SzD0W zlRb`hJn94b3C)i@MX%5|Ke)H48>q#VAi}_3b_?H>M_|ex;1Yrv3nyvDb}%>S!lFuy z^^qML!!`jsRm+X#fOJj5;hrMZ_RX0&I6{K;z(i_x5gp4zW%x|kV|mQl3v|tIuzH>` z-QT`-#AM1d^USWeHPKHngTcfB#eoiD02qu<%{HnUKzHa2(`7sxL$E)_C<_EG{5NA2 zo|v*rm79WA=q#QR^AS4f#W^$hc{1G7-CQAq9MWKo*KMvm1?c=pO@pV@w6vd{-CU0!3coa8R5+Uph)e-&I@-lU ziP>Op9FaV@c8KLDbO-PlVph#6lHel13T!^mPI4oF+5WsGI1f#x#cDeuKBkS}9>}nN z+Q4KB>&BDf;gkBWXr<>UMN^W*t4KaL&g%AOzz#kR zrLqOku%v*fLx4mOVMPuPuu@s*WU8*pgs&W2e>0U`=~5_A2dOs-mQz`vT^<@|@QfjN zQ6W-k{M>n$b?o->4V4z{?d=u0*E{&~!AdZTm+Hc7J@DQCbY;W&* z{rpEh!^?#GNiQZ4@$Ukt;f@AVaaW|V#{pyBN?Ef(ZeR@{EJFO|Ea{^q?R3)S7Zpn~ z9Bi}7I@)+C!^@ia^LD@y16Da@zKm8opC>S$<%oj;?=!TR<6ing)H|Xi&}1hgK?I9F z84lt4ChDhA4m#7Dg{^WXlQ1psY4rg3)uDW1#>ucSX4D;#C27Kc7=;Ua4>5heHG>Yz z4RNpxE_;Gz_5v*2bO})1Rip}oE}H7e5I`0ZC|+K(lc3C9G}v2qn`m-I=Z8edD9Ef=3ch+bQ{09BY^rlw8vG}xcL7Os zHyC23FHR524CtkqH22M)NrTcKt9M5$jWt8Z4Z@6ZC_EW%(f6_$$}Ty)7cFHMn`DNg zG>c%$$4}yh=~)o2KwpCFXG}F?>@g*h7-bi_Jhcx+Os3z5u5>9d4&$CjkykkzGu5F1 z>wp<+LLD{3D?k}gDg!91JX3wuXhISs;cDcEOwwmZ>XBYKI)K_?>ZZ9hl)c`&weJ~X zql^%qA4UTt{+ZOHTo|-m53r#a(uw{-3V&KadyQBZhnS`ZI~?jK2ZD|GuZ%P)88P!j!_B|!xfqkE?aw(GuaQ9a1 z7@Q4b>O~C7QcGHBK%gD{kop^6{WuTYe&V)$=JCQe-ut#EZ|sl1rcx5O^!!_*=RaAs zsMXE>Pq^*fS^QB|c_QIcui8mXY9R!-vBK%Ig@jr# z)5e2yZf{Vp8PHBVLc;4IK2rY+mUD`29QwQ0Y~eY7#UWFHI$N1PVSh1Zt984x2m zfOrM@{BVs#gp=JlKST3Wl7es(=pS}mGJa@@D+z_5&>zE=)>a1BM8US`+2PoZ*MJj2Vg$8`e2Y74l9z&w-Gx5-Ztvkmv z-@OM%#eoIHIS0|~vFecO?dJXhVRF&ZW`)tj^@Ba4T;Vz_nhOo)qWrBu3 zSKhisl5EQX&4uIKFnGhF3>?gg(qh5c5NZ%Qg0*Loyj!}Yli=M3S(d)rP^qTxCXR$` zw5E87^zFi8&H{{^N1=&0430VUQVFA>;set(P9X*#UGFL{qfJO(g7Th^uq`Xf;=?*UQLqVZ)ChoPH zBIsR@vQ%W8=eLcw)&jX zKTdJypvyke6}L%uzkTGq_xlr=f5%KtvV_pCkOeczlf|HXAOy}Q5D(9g=U(1b%>#Y1 zu?`3wEOv}EX;~Q#Oy>}(9T@%fw$n9Q?=a&}H=fw4A9)kqtYh(>(5DlK<1w z14BF3y{(AjeXxmi-+{_-jya9gol?dfBtT6eVVt%oer|Pj4`-ZvQ{hA4&e9!Mr1j1| zQ+v!P5Oi2clb#CS%(h&$dw8k5v#0J(gQg9Qy_`=F0q?e3fz&QJ4LqzU-egl)IHWEp zfKR59$wLsGXg8T{788V+qP&clbnG^91p=|;r#aTJqyaRgxQ>MbnoTnUPfdo&C`oVcEpe3WE0ro{@37jJ&)=U~9rCPvt!V-0#SD?p8U&3>b zv?YI&CI&?%Y9l#pc^>_-Da2mO&ZB~aES=V7PCKE{)knr*iz(}&B`(i3E=c|KrAm*$ zZO_&MOo3^9`I1+1|)s}!jOMa<7u=TObc)^ zb+OW=Xslu!k3y;;H7|X9jp@1xGr%6`h?Hwt@HRx@=-ajzV1EDVvf($g!=@acs)$r~ z+(3EcbBncv>}=P-KorzFWq)y(?Nyb8y6Aspi;Bm_H+sYtnR>B8Y`uklR8J6nhkhnp z>bubhB7*>Z(>uZDLtX~V6qV)8H5NNA1CN-dZ4th(a7N!b&EDgo@dg$TQV!8 z4-2xlO#KIa!-29K#}-TCxPR7EowxNG&st8rmbu0%XtG)$R}8U9_qhmszZ!Mw581hr zey^1=pp`Q|DtAspz-LS=5fd{&U!QysoXo712ZCUmnyulQVylt24SS9S(U@5(t=nDV zmqs$k#LN&5gfq2XdEz`a<3M=8hOilxF0Gr=EL}y@oPa$#l?t6Ou7BBlKPRc7t4K7j z5fd~hfQIFp1^kO%oK4EOY=z^*#jTx2p^L;~!Edt7>&6~qbf_;U#NWeFvEEWK++>3D zXN+KrcD*&ered2N?XZ%aS|B9OD}ajzgvO^Sj1=e*Zx*JPn?W&2&O<*I5GqpuB?9iP zp7@<3#Qc_zCNt1WF%d*^LmhCu!g?{$#Uq%|Iw}hWdsW-J=60eSy~=h{_R>UtFrTzE z2x#89>8Cf7k@RI!Y6_c0fZz=7MGgtEqk>j=R0f&`oV6X2+K}lehjn7gyptvX<;C%KU#YkLsju+xdS?G@N1S{Q z?w|X!U$)-TuBnkug?OLaR z4#O<+Db#r=9A>R^w%F6|@=!=D_e>0Cf(v&ap^K3WHO{Mqb=6h6vdu(Iw(+F3P;AdC znmE7x+I)9$ZOvW8Txht$c_XZ8y2@V4J#wj=U?gc*05hQ82ci*-B*aa*$16Z!C0r&9 zZ)7d3mRU# z+A%43{Puj`RPmc1wi+(xNcZM@#J31o$0TQ5ii7lmABGf`enhW?402;+=S<|l2&Z%p zs3l0^1Tzd97m53{q^HC`>ZW@l-`20}6eP1Fysy$*-q*att2*htuHeUxjUfRYfKL#0 ztyd_QGG@g%VXnzq7~`%Ssh;L~QbZIPC70K4f+@YJ;@e$%%ggbU3Auvwj@o$OO&MgS zR|8Ttg!+?KR#dcdWI%kjt?MqE5Q45{31ScoJswI2Yh4)Xlc>@iZwPg|w@uuj56v#n zx&5Pi-SBHrs1Pv>FV0{p3{HAR-&AApjKd*Lm|#Q5>0ny&*^faAfKylE?kIw|c2Wny z83>Cz?)oGFgf9Ksp>TVwm!w$1nA_9~s(h_l9y2E2)2ITneA@chqG-?NnPu437|l>1 zlwiq)aE53BN;?KrHFv3I^YZ%S@NBf{D|OypW+iKEMc&_A>y3ZBDOa-`80LrxKNVgzoP9h-sVHg7y0)qs7;PTQ;_vfwd z(;`1`RtM6iGjadGu4whlu3yz_zr>c=&h8vA&Rlg~?a4Zy9e1w|@yMx_Yz~&~Gr375 z4B(9Mwceo&s+#M_Pf+pnzz4zejo$^>!FdC@n<_=6pI(LxKeTCXG)^!|;c#ZAQ$q5F zCpQIaNM1~_51k6bVIg96eo8$XIA)f@y^`i^;7XSwKYT0&9Mc>p10_t>)O+xf8#=lig*@8qaNq+%$WxRHi7)2 zP46xOd{F&KZ<%(2yY_&dU}YIUrZ>x{KIPDaOmpUO3?oU_azQ(DJL~UyAfScF-OZX5 zW679-!Cpf`8n9kT_B`CXRe0+rO0|@FZ!eJzo}AYE;l$9;=&>iMw7wI{*wFomP?STG z;G8IJ`!jr0N!PVimCbU-G>xZp!kc}oJK~xXv?%*UyJ%;uTP_1j)q=2_R8qRkG<$Z- zo>V|Z#nV=aXr>=BS|jBn(}L)l zwbx-TNy@|P<&=*jhk{$WKSPWL!c!nAt7m@`d6UIe`dIZ3Wl*}R^X|+ zST|GJ%cCTYCa91=1ROJh!sh@~%xA9FV-z+aNA>dreh(<$YC3oF`~feu(xou+0w3Eb zfEu{UEIrzS!C8MIxoQOOpm|wjmg!`4^$S0^^rZK79MdO0c1HzM_x3&Os{rWn+cfK+ z;AeHVjDk7OPVuu~Y-T^hGa)9w<+`B-IHTU{wJU;@=HRxm`ptZY6qykY{`Fo*1SJhK zD?a{8*FvYRQfrgU2-byGZXXvrMiojL+H^kk0geML_cVklti*Ota?~T^q~(V?2E?l- z>FwS5-5n!z#C7bGTpJN($Lx|m45g1WlGX#dRN^CaTHWmELb{h{JcM4cW~;c1wct{54R2l^-Yso8{pi5Xb~uYgCpGOyPciwh-bjb73k$jDhC9z; z`x){$#G9M)NpqEd;o~44Y9#;*Nme@u$2{^$Lrj|FOB$^W82f!R#OLfOq zs+>S4-w}?~SsK(8hIA0tjs-0N9b^N7Z8x|KuL2C}32{PyC88sRI1AWHBG^i$|CcU7 zSbC*OnSHtGL7Bp&J;GP)xK#BWT5jZ_KqA5!Io2#7bPPK%ESjG1Mo3U7i^o~bG%{=0 zl}LCtncDLGBt;%DFew$^ZRu8UF9XhMrAtv|mLF|~cSMY)pjc&4pHcDTj4YNrDxQpP zZfT)*G}Z@Eu1u-vjg`9&Vz`uy2{`Lk;Og(8!Fl{>fJ{}~=0 z{n+pR>IdF)bNm@K3+=6X_3Yw}8={%6dWdQsep_TN_dccu;f))WcQng)ow8yDC7BZO z7DV23uT^&41z)#^NR;GC8h#?%xdXvE?;2={ldoCa$J3CM&MUsNL$*5KcnwNZb2OW0 z5K0RxhtcaN!<05(1m||sO-7hn(2Bj&w0T~n^tU91Alf6Lj&q+%IUK$sgAW<}Zk^4& zs*LO$PW7{vI7Z{i02wq{$eKXL=eAVU*n==-K~rg~8ph6Fg{RtBjOMb@v_F;fh#lEq z_Q{Zt3PYMS@Dfoc6n!=;1|u|5nh*?xO!WlIR^Gy4_!wL(e+PD2i<`NtMzLeYz;U6$ z!?$Lmgo)|ZmXJlqVCo;s!i)6`ge-A7#54X94a1^R)yn$Xod=8YKhX&nQ@msH2Thu%qMB1FU#SFAvF%$a3t4#B>i!G5! z>eQBI`U|k+ZlrgwcNbuhMa@tN1Q12&#z`cjQfFVK>HUv$%W}(D->XbG%OUShb)Tm!M z7%v`dij((K9?sTIbShhdf@b0tB;6@V#!&%+|{C?S<&gj`fx%6JAJ0Fg};v zQ;~xvtOh2W(BK>fue?|9jXHrIhTX!qW7!FN!voJh0~-*mIIQwGcKA)#Rp)gs4axj zwVkFV<8jD%E-z2Tr|Cuq3&J^>qpi{s0H8LuuY#{*%>y5pS5NudSo1#zuy)FlZO>+I zK6tf!jpeH0Y0V`IK`xv&lf;ts8*-!w$v-Hv0aP@!w<<7D*bUQ={39i$X_H7)aWt#r zuG%fs@vI5ITg|6hF>hzVP|)gyj+(A@WolR6WoEj*gJSI}2tVi)`{2lcThh~G1x0#2 z?k}U{J!4LKlL(#GHCRhcYattn02Q+}V#I9~vySIS%g2zDnW5SZTtsI<<1{?miaAcv zS-2=7JL@kz(34#6c*5J-O;Nsxvdk5cEuCl#B9?LLG%x+^{;h_fDf$W2{d|H3d>-Q) zsI^gw&1k-4=$3?&mhrl~)rK~*U=`qzXVcP!qkDHwjlx+m(2ikmm3;kt=W+o(u*f&A zSrN!)M}eS%K4EWrD)LZYIz66ODX39ldLfF< zoZ(5y^i8Xu%g`F-YB(knQUOTP{0v9pF>VNQ5+(_T!6Ld|l-kAhN(I7%5HN^D1GR^h zt|S5*2AONK_TWuS2%)@~x_=G90wbG_qyuHD0X%=6JGbY)`WwG)U%lr~|Eb=4{$KwO z+t==krFwUd`}jE?zN%2-Q`|4AEO9J&GB;`m4=9$fH)-tI+mxpb?%&|9AmxLv@vyi( zyZy1nYhJuH-gvtDtKsb`)-cP-V9$|tXf=@+bt~8-EQkEG%=v=gqS&tzsRi;0uGgc( z8b9LHF=#U%VHt`>C}PB`7L~6$7P*k!&9=K(zfd+X_>kTx1Y;aT5wlTP9VZHb?gtTv zaNJAw3W2r0N{gU7}91W$k7d8$W`{kjY2d{uJ>GzVahPPcL2dJL^?LB z+uns_^w5T8{8T{#?|j1(aU#TEiF_Us9F7iG<_`vQYhf@ z2*YTw{{Cs1L^<&tk;zPu(-LAyIJL!^R)q9Ua{y7Tq=`?tUC+0pms&X!6*y)OTp4%( zty?Fw2Ys>i90fD_)5Y|lyHd}>a}TP;5ESdjHCzSIcc@iPU8#QmI)i@`Z~{|OIbzJj zSz-Z$N~jEQAbG%;*{oIQQFvDIAGw=s!4sqdsU5fgemlz+Xzr(Hf7wXR+V52Zld6ny zl~2KJ`YI{cYQgl#9HYQyERf(kz3j^Zq!g25;xbwp4Ma4VsTIRop% z{5mV(p34R6H1E61wwrMv?R30MgZ`R)p0j3@TaZ2F{qC&>waI#ClC#Gg&^ZpIAhOP~ zU51ikx_!9S&4wAKr>kTV0V_f8E@dF{Fr`>e4257LN0SjoW=Yv~8D4Q_W+U4$yLo#c zuTqsW3uUjL(jov@XURecwv$2Brad*QZnMjUkxQaKUyXpeC)5_TuNhU}o-z<|LuC(! z%0L}RGlRf34zOktHSbs^QGb*%q&F4xAxmWK3bZs}rxSRLx<(#|&~LgNnb=~EKSGCt zSzr`rh++6R7`$SvUm!>XXo6%`j6<+YR5u7AtLjEFB+0(grBLS!f_UURqg2tgu?Qi;OeE}!N3t6Ph&FQ`bt?O^ea_wc)3 z6^;Dv-Rf@%4#4eb>ofkJ{oOkXL2m4S#J;!v6Ls0+^;eD;)fcpVU==+k05%d4?p%I> zJ~^B44hk2h9fFZL*v^f%uSD5^59yDmbK((81$YMvS~(dNKg1~1fU-S)u(eYzTHJNa zF~s$Y?FX98kLZm%7#;z#MqOSak*71bNj zHTOgHcp-@(bVG%DGhsq%6T6iNQ|U4LzHHlUZNnS+Q{ul#z?e)aHdsbRHxnc~Hx*C& zUKvrb!duf=Ua%xb!9d!z(xrrO&m_Spm@I6^1ltyz^@6itIBQsoiuMR{n0O7wnpm8=68PM*X&Exc%R~a<$Yh`Q8df4bUP~S z&fmfVw@1FX|CoKHtc%;%uH1Ly#(UnwtNW^~ispj=PzPk)#)#Ef(peI%3V}oDj>!V~ zqn+tgY+sTz<^F>0tu|Q$l9C}{nqi9t4Kleahh@oPO;QkEbY}Yu7VOuA`qVhQFeJ^U zDZ*yaI0%t`?xUqd`mJ5vh{$LPnw{P_b%J*BjVL?;Td{?r8B)g0BjVIpc3^-V_dqchUDfH7wg|SJzMNv&0K_>m zxeSq7q?)^OvyS0K^fsGp2%?b~9LlRoHLIHRYsc!qbx|gtNB%e4?X3!Vke;5FBvtsy z?!Yw(Q>1EXzvUU`2*GjsK$N(l|4cymkgU#%*8Ge)Q1Z>r=Bf&g+f|puaesuX1N_?z z(X7|py*P)t@EA!9Gw12c>V0k+l1GVVR@@iRxH}4+KrMS%tNx(Kiy2x+zvb^C8XezIl8$C;bb9atnvd|Xp z!r37IycD{6HyWUyQ_GKp)VBO+fl!n2G7t*vIMX5SM2C3?y^QoGVYAOzD1txGS*j(8 zhB>TAiuzHQq>==UR@sGv!r4V5=6ETX;}naMG4~*+EX*V~7lh-Q@@kx;#S3);>>D=V zhxdQLDXw%W=+z~YQD}T3pNDFA*F45b(z9?JBHNDK$AUV6QY4P)%U|%m_M;Dek5}LN z2_AwE?mV{p2rszZZ=Y02lv}&8`@;{|cX<5P2mj^IsXzJQ`|MjcMoJF2y^s6z+#j-E zO1>)p*uwd-EY}~V4`f31JG7l@xC-9PR1{!rggZnc<^Y}yr6XD$B%6*OG-t|8=~vCR z+7o_be%CmiX7vH!a{605`_6g#OKFVSg2QlW*yF=!} z3`$Adla{bxPFs|A`z37T5>MaSHZ^|NV$}^@$5ep|8k%V%_+0PAS+U8E){==luN-GR z4LLT?3Zc4TDQ|?GmKw`FU<gAXfZT*T3=W8PY1 zSsdC4#5u^d0&L01s2N1y?{bAE(((A?a%VK@y`V$y#Jm~8n-<~lJEG_|L(@FJs|ei zA{U_q33!VWfS)B6TmJCl)#(kAAj$`TO^SPBa?ODo;lbg0WF$$FEBeuHki$q? zO}y0b;RHIifDV8rv-NZSCQ=(3X_J$p=t#g!DdWtfztU+zKYaVD;g7APKLO0hbxY5n z%CHcpYuXvKl~P-S9`fY_AV7HG;7g`qmEvOX<)lnWfize;D{w&&G*8>itb{f#;8ESs z*Dp)Ppmny0aA&nJif2rfqNB!X^I6*vSar3&vCkOC#&ZJE@#88`gfZ0T*r~_eR2N{+ zR+w%Zh@~@#+Ta~SuC>%Qlf;IIl{xKDj?uA}(sOH@?L#(UF4Ea$=t6MWhuLd-GDCvS zTER;?Gz>`PdcEWWSo}_h{k;rn%YOGl!LJQ7`RiO;|JEX|=x;kf?nYS11mJ8Yd}$~W z)`A8rp$=?I9wbhAt3gu_N4fsVRUCI#h|pw+T@2ET_2OxPaJTomThg`nHLYX))B-W&%$ce<`a5fY zpnu>60KFqO5PZ*AdZFHrfxi6(#v7*DGE)LK*4UNg-;dDCq6w^0n>49R21d zf|r*1Cwr^cH9~*`QWWN)CD+K!q73$>>~gdh2fh@l9Dbo-({cgbgPPA8uKB75L2GDk zMGx;-GIBajhpb9xo0&qBE&iSl>btyyUegKe! zpmFL)1QBt-J`3RG29Si^g5ePSi=X>N8^lbP`4;0xX4gBOQ4JV3%t{bo80f>V-5j+# zm>XBO)YZ|6b2gUA{@W%VUFGPsJ%u-<1hr)1`)MS|yOR;e77-scxNJGnn9#PM(c`u7fg{115X z^!uLRk=yOB*k}9r@mDWC%&SK~t@hpbKP3D9UG8!OPj2$?26uZ?io;F+%A-ngj)h(3 zRxt+KFRH<*-}@_a7WSr+{AYhq{T^mLp`CKH^lI9MigxL^1mdgmfFQ1_l#?X6iI71f zJZ=VOWgn}Nr+%6WDm>^;YV-AFLZJW7fI)|vdCA`U9{Pe{E_+!jQ2C&01Zn{h0r2p# zvQ<)0)sy@T3AuEgYfi-~Q;-Sj z({s!gmuTMGnnONj{}B7)E*ZenqzB?m;SA8kdcYXN!gX0QSGp2w#IDRib{smfN7Kxw z336ShgsK$q`WFoFOF0WP@XFc%@F1K3QBa_gf}k&4NBUK0ZzGM|$6 zFAx{D-U6|8j^V6mc8BTuAhFOBJCMMIr@_)lzG%EI*?Cx)Vp{>NK^*3to9X1{BpERv z6!;RK@L;;h065um=-*OY>TgUkPn!Eapre0)xXk^So6byVq>!rI-WII!pPcB(; zmdbkARwSLu+r4mqlHF7F)H$e1cDp-m(_k8q>JVLrQkV`=8pxQBpd*Q~O$fA^6VxZ= z<|3VQrz$-*=5A;}4GD}eHNPGr?0ktG&9+E37^7=M(u?GzyTt}PYX!Etf!P~R@(2^k z`(O~MoxpO>C_J5S2ISLNG;K%$FRxKCKiaFi`6oG>Me|!Kax?zl7|@05Y6y}O(Kj;A zV%5~+nQXU|U|dJW*2@dY(9EfW%!Jl;yb*4j8giL z^OEoy44USx_~;Sa1n2}1ZWG=%TD(A`5J1ZU>QhBHO)30SyRYg2=KImzrWt_pd;z9M z1uUQDFtK?MGgavM2wIhfr>6e*5o#zQT?K|RO{fe88m%r-5kqTtoR=jDcvjg*dp#+& z86Yflgf1>{>LXGrkxfFPSHo>kU7sQAX#?JkIIvb3oz>dhgms;SWv{T_IALn4v zCs{ZPY}2y39!S(O)eqpok69lo(WvD@LQ5YiBhbvy<*Ryv_V+CjjHGDA^-myPB&>xk zk5yHIJwakEgr~BxKGkRKEp<=(he4BoVNjZC1cf>7a!PRvw6l_T)MXH11mY7EV=CGm zu9igNnBd_S|6$9ks6@(rab7%ynxi3~i^rsc&{P%hN&i8)yTQ7Zk z^U8hI2*ww9@BKT;b^Y0^p=xgS_NAk5@nTzuqR-kZ5Z`G?F^p*?!-@<2gJqayChhK_ zY3kTUh-|bfUrLLl%0tg+%e3$p97-mnchjV*!)_+ccFNj#*6v&7v5W9rcIt(VeWKDK z2hJF1NT*DLT@CjQ7YEE#EMs5e$e+$YJHj?nK)J@Jjvax$Me1TW@rlIkf>uQz_t;kc zo1so?l9Xyjg)1=v^-c;9F&Y~8jD}J&UAtp+(Gb{OX%Kx|lybee&G=u6E7qjfRLAfCk?WeXF@bs?J|9MPP)`4DduB3lk^Yftu3_q z&09yuP{m(s6Y+j$TO3YsN%y1yINz$lGPq2HTs=V!=tbsRnF7 zp^==luxb+PX{{0wNV68^MMGkwcSCyxU0m(E#&?P z_eXz_@c4f2+@cC%!b^?4bNh7q$N!Cer8i!@d}E|K6}J!DC)Jj?eNVr`>58>{r+zLs zRlFkm@{xDh=kGkk^Lw{=_*V4DF;Aa*;7b`#Us97|>;88L-ua-aRp-CT1Gld}{RjW` z_r0ojf7`xiAK&)CZE3mRrcXfiJ#H`CCmy~3_iz5}y+^nq67@y&O*Yl`5t_kWs4U7o$Y zgQuVR&f?49!5{v!zrFa%eev?|{rc_?-8Xps%&pgbo}Smot?t&#U;BD?6zN~Px~C0o zZ(sbkf8%fQ>cx8%Rn{a}igEg>vJ#T}WCY1Ak|?01=?YT%MS2i5ZU}|rC|5r}P$>8B zDacvrQYM;5B{r+#aRM`Ma&0n(dKk8>GbjWFQC&9{phXz5Z`!5$lV8xucueQw_x@8J zK3MW__n-gAJbs8fx8s{U`OwJbol)@&zQn z4wGKW@LIdKI$0lla4*AZI!GcW3BlCzpejW_@YgUP^doE-HIEq9FE@B5X)KVNj1my{qTuxWqZcsEOfUE6@{EoiB@Ot_=ABNuG!$H>qEVx1^P4` zTOzXQfboI5qDYU6AI3n4L!*g5qv_R@#8w4`#c%wVk)o`0DQTkk*lp}jl568J%2;RVJhS5%8(gF)(L|?rgZoo*56`}} zd`$jxoBMlgpY5zpLYIMt*%_d)wLkFVUlZ}YJi-^(kx zc6&Rwc*ToH-~VL=335Vz__I8Gg}c4o(6xL?k-M5im+RxMdqgy7+5vwoL=B6TPMkk!ZYEv7Fbktv}Ppd7dozHI14?A`k%Ej?1 zU4Nz!#$zVLm%2a{p+$>_r-=I!8#);z!W1MTPhKMD0d+|pCa_A8(~g-qUp}w3q8Gn` zdtlMgAoX)jr{TEiFV&OCtOrms`ovF^#;-Ys^J396>f`6b*_jOm89Lcdfe^n|wfdKGg~zim7Xcfc|c z#CBQtYMRTJhA=R%=3%?L2?gr!5IIXJ7f5^%fqzJ*NOe-8%_e@rOlH`DGSplyzqPwg zOsK#pk{g3Bfgtk@l}@5-_vG>VSBx6YdUo%b0Yo6p+ko7-x#`6XSP7=A{YQ;@soC`V^Gg|7mQCTgY>YQq*-dbZEz zSz0x&cKe+eOWYcMVU6?F_WLBMQ`U z#H=sfT55wn^8$GZTind!{~}2vw%PI_w5dP~9$5Jy3$^)l70nX~@|bcb+h!oQhbKmhj8Nv&-tO5Y`~mh4V~4v$co9vwUqAjp&*!Att+r6 z>sN)YRKX(K!A(FRdQsfG0KA0cln`{gxnLM*f-?ODtCCy7{UK6-X{5T;=AzN)RI#+V zXmk9U$eP7N%s4AfxC<~8cpPjKN7I8gnpUN_Pw&-~@bjy~sb3uW*_*=wiKz<+P6p2f zd*M>z96@AW(VWN%SSH||o8MBpvLrj)=nXf7X+~^JfppWe(|s}TwOZqIJ3ZNh1mGgP z5_5F#mxLR?ob)rKB|?n!H(4V}O7M##?FdUNYcx_&+S~=s37tahqNj~Ay)PM2Y$N`aKqk2k3JLXEO}E3 zFCkDO#<@$NT7e7Y6N6T+q8V~Tz|(VcUZ(#XM*-6e{|YfVT;j^f)VN^%%uN8hAoSM| zC#Ae75Xb^VB@vftxZ9+ZwR7wN$JQYf5V(ho}q?w#*^rv6{0;P{NZ z?CZ}b|NL*?s{6Y9yv>!1G_N20m^}T;%)=|i@R=9lfu8DVga8}tu~Y+ZPYc|_mX?xG zUKL*qz(NU;qMdEH>8oR<;fH%x#=CLT*C8DfSFiH-w)ynoZ3qT^HNcupzICuAbIw|( z+kwx(nP~IYrOOCA8a5lF3PfU1`D__fiviktOdKOT1?||p*-(NRbC$LC28ZLfP1j6R zw3%3|T73Vt*IX6{ifbkoI$VEw0_lWW5^Xy!r?{nl!#awnVJ|eS1IVNT6=SCoc%dW5 z-eo46i3FrHLLylBA2{zF6!uPc>IGPuip$i5kKunJh&gQh^r|_xA{p$(k}(AM1jN3! z`Jfc0t0{1|w-~Z1qs8M1-AQBUBixaWi5oMdp1};!LRw>_F-AMQU^W}t84MyGJX0&Q)ElBc#`t*6ajikH$_g=0?9du!R&FK+>1~F-wEEtyO$2vb#?SuS zU;n%M;RWs@0or`(dHuxi{UY}dd`ABu5Ed?_AT%!h@R$GiH@H7VfbhfjzIdTFH{d^o zPUi3krYg0ucbUH_Cz@7R;~mA&U~a*@fd~-ksI5n#t3NP}i*5>DD&Nv}q%kg+0HpOv zWN|#TT#hvhfLhV>aYY@w0*wH_u$mRxhe--!6GugcOH%SX#>*W<9%5$s7>e?{L;9y( z&^gTeh5U8#e_6Zo3Z-5dZN1vku&a6Y(@8((5zO;KCWE@7*To;qq(OIE91t=re~$v2RE~# zHi6~=7WN0j3ST5Mo0NAO*VZ;M0s+xYpTVZ8)PMU|%R@BQ^PrjX1HkpncQD$K708J2 zEf9QVsK<5?qDXCaQlwfzoE1!=5}A=U-A+$v(ilrdFKpAd>@f^8B4Zk_xa=Og0H5^X z`h`-)5zWL>0Y$D(Ck)QWTDDgWsyKumS&@ON`CwuR+Mlr)!nh!m#&b?0p5~AUxn6|y zVc)JYUM$Xj?jpFpof~V9zfVv-TrLQ*h>HLNpW~Venoj`5T!NRsbbsy8+5WLO^me+cJo)7=S0=-1Y~F) z41&M`abtW_&A_>=p{+IL&mAAsAb zV_Sr?JRBh!cJ^a}yyGHJ90Ak_2)RB8|Li~H@mIKVx$=&_|MUd`=jh8Mo;|G31}4Jg zY|O@ea5x}0$&)3-6_-sWO|4A6CZ%dg8ZDe7<}p#!Nev7U z$d1x8Fi#7rw#*~(>HDEIvb9vau=HuUeKOB=b`c5D1BizDu@B7hEmXiDB#}U_-N+v* zSwZy4XeFgw%4^vuV**G}g^Ao$PEG|RV6c#+?JP@HOTD2V(6gY>W=wo?RNO5$*npqQ|+0*9Ca0FJC#*_i|38P_-^d z^C$j=WRm8kM!o5$1e`8#*Pqfq%E8jT4mnsHCR_|)5M5wBlwn;JT&lK{9J1|K#PLkj ztC})3#c`>s02r6q9XTK~MhacWg090nf~w@Nr3ciDYi^Q(N)|QzMbq+t+5{EsOGUor zso3@KQ8`vnDjEYMn2=9x+<{strqNF zEeW7d7#J6|(31A)FUU^q7lI zg`NSjMg#})6qrRIY9)1WL>TLqJk{bPwFOn5l21 z;>Y#n!Mof)@%rRX{-^hd*)lI5zZ+-#{5kzxjp4ofz)$O+-+EQwfA|aBKk+>Y5(HGs zMZ)GU>xZ-3FFY+(`Oi(>r=OX9Ruw6q4!7cdN)1}9juAN0^)RI)FQzNq)7*Jm!UU=z zf)EglClXa|3FyJJQ8N1bLYeM18DtYn_tPuW4~f-Ct~D~ir_)gd;yzEFL}z1@3Eyxu zvQ^sYrc|IKMZOQl{2)`()y%p|n}zfU&@TD%oPnaBUmr5VJl(;V>u0Muo*PgKLm9d3 zi9IG7zef{4w# z9}>@AT&QQzo}V3z`dB@}6ZN5RHW+Gqjqxt6JDLKuZme*_AeI)lNgd@#oB)W}YJ}g} zLP+K*EoJz`XiU+EdWRR)K1QcON)@6r0oIJs_s^6d-oAi*9I-L+(cXOA?8Y&mbZU^U z?Qj$+Oc^q@gd`@V&>Z3lI3g0Ej0nm+_n=JZq?rdZ?6;X>?f{FaRSbmzh}Eb`m3>m1 z0XoE$>v`XRc@ZqcHU4?TZ0--(IcWOdkl#HNQ@O4Ih(N^34!%kjSA?glXNd#-&P{OW}{!ljn6c= zr}pZg$;7w364f`UQN|8p*An&4wQ2X#^N+1+(Jl)Kpu8_T@Hp9O^ux5lh6Wy!Nid z&42QDxOrISMY&wt)b}6PclUB9bpS3m4&MHQ|Kyr}%8N>5ISUjC7bEb%P$lPVLRCrR1AYglZn}6=4DKRxs z@p(20uu`>K+ouU`K5%ei1#C652pDV#KbB18n29?$QpB}D>)BjCs$0|}vC}S7X_QED zWw4k~%wQ^wS=0(Wj85VXB(x_mo@uZcmdN*+X0=-)5Gbo9qWxA|vt#y7&I|q25+N~x z`8)kNJK<06`b@{;yy#M8nC}dM&J9?43onM^E-O18t(OH*zU3M=(=&gTu>0-ZP7?Oj z%8HjEC>S5EKYcP+;uvCGCGd}Qf?{|6(p|DHRRuEC16aNhaW5E17?*MgwnrmBbw()@ zeHc{sr)jchlnSQFRtL*;Xef9MHG}D=t`-|@i&Fq_a9xX2)v@_yhC$JpOWJ#%%yl#^|1th%{Cx}C*RHo>(+aBZ(j zuZ%BXP60$lhhtTRj=&6VpIEz_@M$hXBqS+l|9N8;JIOuhC#txjW1c{amqjSsp=d1{ zgav$wofKUQ{YGI=_kQq=N!wy&oLthZ8av5dEyh3sU zGU?GM%g!`&U+N8M;XvJ1>ut`;Wf36&6*GE2fMOdUDiAOQM6D$qWb#8;?Bvkw>kid! zIi*~#N>gu5)fB8k0q6*s)3e7UD{h&YTWXhv6V{pdMYJIf*%j*FY8>%^4rW9LN(Jdw zPfQv|Oc#vibueXggF>d4BsN!^R2ATHaLpqA4AhYd3o7pLeWwXJ9@NHqYUtJYKzHg) z^8iYzm06-VGO)*qbXH`Uy++X)R?LI-ZPW;>ao(OqP*bQ+(f;V`mNSJK>O{@%WC*HY2K=hj2+pRQqegRsdWa z^3t^3R&zra3HA!)+)}PxX5=18K0qeW<{yF>$_zs8loK% zg6Gl=%Vm4MeoB69J9nplzdy;DHTQ0ElBPtXz7fWr+$8KxED-NE@FvPKwUf zsLV^o?!7ZLs!P2glQhxDTZ;_xT6z%SmLDP21NtyVbfQVEG}1M zG)05fZkIpCum&tQLie)gds#N%wOlQ?c?K&*VkIItO`_0hOrGc0J(K+|!Q*0!I#6ke ztC6S(QNm1wPY4r$Apk;_=L$ufM~HaOFVKED?O~;!Lmur;r*-JcP9RkVGi@$klAI|K z*mKj#-E^*)$0D;h&8#Gb25P_bOx5f`s$@kHQ1qZE2(JdvP^1Lj|nLex5VGJse#A>N!lH(`WP)6=`%`1vtx?32~*x;x^(~H8{j@nm9X}T z(V1U}!WH_4{-#IH5ZfAr*bcyBwCRGBS;SCO;+P8FK;kfq_`-a= z*O~gKrQXm3xfL)Pr+Vb9xAYOV!SVRcmdxP9l#1%oAG2uD+K0Yb*ZA;2H6^6>OP~3( z-+Yos@4s31zIiOfSAy(0xA#**KlRRi+?;R~!bkTxsQ`)}<01oVk8=IpzgKgo#y2KU z)@|}lX~J`<-5r#>G7(a~P5Dlbh~GINHJ2|MfXd83!}ubpM|DiZEwDL_d8CRbevy#n zt?^+qE5)#g@FeV~Qao!l1#8Q1$h$NSbDMD@MkBO6XzwJd*PK1scQ%*Fp`vBt$owWR zGpd5FFW9Nj3g)SpjLaP76&XPzYXmoh3B#=U3BtJ8Fb-|UBS?rvB-R5}QHx^OEy&od zfKz=AS1Y?erH0bz1zzRkX$^<9E-Vo9YB&Ety#JVc@s^-ntkp_2}_Fhv&8%qQkGkmFht{zjRrqWg{2R?N$ z0oOT??Wah%11Va0up^(by)?|;F2fh3?Nq1y(6&udqL6|F+Lmi!Qqmb>FQMuIL{oGg zlEkeLGl-l#$suTm;`S_|1WUakRE$2=5-2A4Q?zZ0wjFOPR5NMlv7va{NHJ^pj?eex-RAH26VNIbms z@CLVBKB4cpee?Z){*SmjHBEwT-vy&9r4PP4Y?R!g2>Ahr@m@{FP4h_Gm{gcDQxgbYubBiX75heOo; z@V0FijlxAevBAYc^|SI4WK@`vBx;X&gvX=XBcWvpkUa0s0xE(X4xhQNW@+{*YKW&= zYbC(G#1gb?Pq^A?)YA4njI?UOENKpu*ukPShXu12E!`pvIC`v*@6gl;@*K7aDYzl- zLwFwMq1$P)3TQ}pv|va3G7+?fI;+RvDOycTjZ?dcncwlK_##~d19=&xC}Eyyw6N40 z(iR^9Mb@3XW| z{Ef+D=e4_k&vW{|cJ|%|!T_&5dgZe0h1UGJ{JabNTHL(kAP1#V*cHgZRMia(k1lG7 z4fjBkTfBl)3S|ZL?@oDg?`95Mk$3>9LAJYWJzHjh_zsgc@S4oKa?v~EU)&7UUXK;wlGeo7~8cyv1<%fwo6kI#GajuvV@D z%Z#cRfKXCAW?+0|!h)g$L+Bm9E>*A(m_?Dk-T`yha_5kpAQvE8tGB2LU}Lo+i3V(d zh@~H^wv;v}s@Bw>c`zEi1l>@V{+r0sIv<+^@md1k4|)~QR*B>rsU6JWGU9&KwV$rwKxTgJ-v>uzlUeZ&u4K??c z&vnV)mzdb?BRE9bgRvGwchk-XU@aaAd35 zmshA_PwoO0NY}4*4vy|xb9{D4{Vd*?aYu=C#UfIaLD-;xmr|-QB#T>*O0v#;p!!sn zU|8r0={!g9A(vzhF1Z%WRX{R)&l`ELcV&6U*Z^kk!SSAueacr1t5gf23dM$1FpaQI z2UD_(cE|hsq-cwXHPxsDXdctX(Apmu!$+XEw)^%);*8jvs1=3PsER7+LxkGF1Jr{q zBt^0;%SR*BCxfC>`#G%fMh4P3J<#miyXjc1lqAMxegz;BJHS%(8)Uk2t96;4noQP3 z&(ko*%5hyx4`DPcn~q)EeUW@GVUq{cVPR^bgaR}yms3ap%Ff8gZ>Bmy zXm^?iK)eV@OC=wc9|qk#dT^q}iNh~oYz3PM|5aRiWtE(1{JYc}hDhKs6+|)Ltd<3| znYUnbgp-SI?yOoCu6=Qe*WvQAu*Pw@_TG(m$KU@zaxRn!_AhajdcM9~`ioz?tRG$7 z_}1_Kj(+%)+zB__)d!?he~asPx$a5fZ7L!#TpoM!K!$#~)KfiQyL!)!8}~^g;GR>5 z=i%@ucIlEmj!KO#QV~0@Qys~;7}`NZ0!wK`&)3To<2ggRH?C~}CL~a`V9>XD&L%ue zanuQ+@ob>mt&9+El2#ykNekfVC4v?WVcLt-=Ho2$)3_oBmp3*eXmRK2!x(lapF*Z8 zPoBfz5S%H?SxE)I1s#GFa8vPC^qU`%5MSwhVW>|AI<#sqs=-Ci*&sr+MfETYwjG`d z_A2vSFaSl75fdbU3P-76hycM*5uw63>XZ?yIU6ps1NF6Blkr_1A;LC9~R)cV6u!Om`rx${`UuipCoaW`;36!{SxlHev z-d{0PY?F=kvxiZG6W`9~9$G3jpWI(C9AzU9C0Q_EB?6$Z7l{-olPdHzn0faoC#Ic( zAz~!^vUlcJD{W>Aj;4i<5LDFJf=l?YVubcW*vc%hFH*gnz?6GV+LGqN)l-Q47Hqa< zqX8&fW{OSw8rLk*QU~#u8TS1sH7#WFF(D<{ikL>KKZqG?|b7- z8QOloe$L-Y^n>wFzbVeLp5@lhaa*%Z=$}9HllSwWcAX#4&ppE3x4-hque~fhG3Mp! zH)}6QycpmG>o|&yNU3V#yZ=ai4?HZ~l}`#eLq512-G zpNjE8-4SO1e8MD5tC`e`u)TgVCED!iI|7f5h8^by&1=yX??|Uf(-gX?#163a;P}e~ z-{iO=@liym%81%@>-eWzlHvL4tp7x1R(3?lRm2ch^YI?RShXZ9%0J{l9mwW4_j^yX{f=NJ=U z3Scts=`AAj4GE-}09!enbFt|VHn|;@dinv(0zd$7D~JPgLvRXf1iwBV4u}1c0vDvg zoh2~5K~dI~YtZ9Sb2jOene-I8&P!Sy1LFkBV4mWY?T_n8FIJ1|&yT4oXkutHn0N+v zaoAty6%6{KDpiey>yN+$T1f*EDrl6~fs#lBhKd6S9GMUR`t2p1Ob7_C9t<&~)fzUz zYN0IOGE=lOY&^Hr(C7}YW~PKc`yd6UbbH@4RKS$hFOq@L$*}~huH!EbRtMsGXaqBc zf1(Yhe!)Of`C{^1xFH#G(3~HO_4^ZOD1@kQ3OjTag+|2TC>@2A5MP zOq+IGG%bYHV*CaXz_7{K)rR%{HxkB-7^2m-1rKHA!ca|s6#)QnkRE}IU_d`@N$9B@ z=prEO{hpSTL&ZRU5q1wqkYG<1fo)NK5I_m-Uos$(pv~aKNa<+9--q}&9r3vZI^37~ z0#BOfkI@pf)8J-56uu#2E4g$U)l74*nPMADABcZ$*+4;8IoGeSl_+(n4^l@9atJNW zJD3^ZC6c#UxKc}JI|*G~kMe`>P+>>qf(7)XBNYVchJ%`K7^E;6=4VT}x6~UlX@=b9 z8W3DFR)DTGLDwB`CM0N0wgXewcLh%?)ww#sz%I&|$lh#3T;M{yirSiRuhuC8rbUpj z?TNfs#yNntak|fUAzw{XXH#%AbO&^J*RBdN?Z{z~4P|rA6xxK)t}~BEItk$bY`oTH zHPHk59T0TjkSP<1eQqm))P>Z}=zJ_BZXO)L3vxdQ13ayakFqWx^HQiLHd5GpGw`s~AUkY9 zOz3tU?|xaM_7%z9k#LS0XCIvaRLJ>id(S>Sq7|p@E{$*ZvHsNqyA4HId#z$ZzeQ;R zMiM)UdFaEHrBPNC5prBG#8JV|K)x~;FPb#4yrPejiM(2Qs;*+?*!q84CXvejCVHCjsiirUqBYv~FRdKT2Jdan+Q%85#(cY1uO1jh=0plTFWU zG2!Z%w^KBhC^)O9jY1_whPf@4P=4v(VmfwN%_CnC!Ia4kGlY~P+XB;K_n(vCl60+~ zkdqPi#ivuXCFdNo9V+5D&Kkf0MjV)bV6d3b57@}Y2p%+YdX$?onl`BNoM{rj)Ei?0*x6(OV7hL(}x?*_iq3^u(l&AyJ(JW@Hfs|PTpp7vaRC`1Pq&W~ z99=*WdDd*9bfzSh_0ae&Wu%>9IIglUY(V+UTfJE z_5i`5VaNGKhNf@{WLC@3FEE=$nl5vs&;zJ4Q@t6l!ywUL%3u+ z28u8^h|<87Lv=bw$elp7@aaU&ty@+?Y0$}~YT`Uzn@&y`@Mwcc7!YTSnqS(S-ijm3 z*MX0f6R?nbdkuwp9U=%?|0tc*CF}MPMYL)Ov#m1Ew%HyaJ0YH0D^j!qnf7KHOImVt z$v2W5s$kUOoz#IPje+JfHQP(QA^shZCdrpU{l@|Xtjn(t{dnChI+4B=7y0qRU2gvJA2;$?K(ZvMn?VJ{meU)OnPg%8os@4M znH5%X+;!DPc5P#~#nm64?7c;gsHe>8h&-(_0=7(!pcw`g4>3~yQNwZrzS(3>!;7sP z0BKg0j1p&nCWPGH^{5g8I>M{-yaNZLU-#2f$ zX|;k^t$^E6kb8=8MkbfgBbLwFK2tEnEaW*OQ6XBgm~K$gWyshf^TJpkvJ$3+qO^cG z$d3uYov47?nYT4fH}q6{q6W{?rhkKmhcckCK!FF*qGf?}%(z;y`7&rm*k!sPE!lYI zflu0ps^Av-8wJ}5TDX-Dy;%w+C7m%aIH`vqO9>^cmsO+E$vYfpYHDL{jhM7Q*l87; zfw@`(YtG;5NE;SOMAgoAdu!RC({knik zQ!Jt`FsjnLlki&hk&>l0wowF-UukUid#{zr1TT~dc(MNMUaBVxwC_?Ybb;CJ9-tqB zeUeV_We$}B1z1YkupZy{MGKc&+KgL!2n6vX@!0?IN8+(Eg+m$jN8+(R5|6EOkFyew ztxZxeZ;}QgYj_0Ls)TV*0nTJHx*tHrh*2lT_RA=&sA1A8J6k>LrQUD^8K38`p#UV- z6;KbCZHDkfM+VcPd1)Y3;^D0^IQvhkyAC-2c=LH&8Ti%jN(3&wfc_=EpAG zxKRho0w;d>9&QeQh8um^=mydZi(@7~YNrN8)9HP83>_wG{>`sZJNp#CTkC42(r zgR~f{egHU~o#Q%1I-DZug4z~NrFMEk%B+ zBTY8n1llj&+JnmUc2AJ)jlx*PC-&+HaNki72-CV5VzZ!7=SRMENT1n|I4rA_fDu>~ z{Q>$UY>f>Wk)TCG_r&kSj}2Xl89X8cs-lwPjJX?mREp+|5e1+fKLCWJDKDT+{=a3L zCs)gHv3?cJGI_J+xRn!N;)j-sE#h%BIFT^R4qT(GRW3|?gx?!(fqV_) zXu0oory&}V75S7#nWSl&^7Nfup09U|Ohy`u20+CQyA>eAYSpEg%Js8wMzz@=RkLn- zxO?2Q`$iG;c?%Yff=w561w)}l=bAB6DDR3Qnj*btp< zxpt{Hq$Y%MLZGB=oPv=^0x=HCP+QYRr8b}qOkJ{d6*J%IQ{SIH#SgD>{p53xarS+;cC46JCn!GeN^Ny_1nrj2SNk|EIX=}(kTGROm{xXJF6K8IUnDl~1GhL0y&QJtE zJ!Z&n1Beu97&sPcWf39}Db!aEn&;HfHc6B9tB1BB&L2wBOx zJO?}NPDv8BuhV2B#Q>PWDQS>s?R4IQEMQTd0g|83f*I=TI|e~m(5v=E&03m5u*zmJ zlCU6LJw}PcEGs)QB^BOeeu3)YYk68V`wW}n#)BO+y4b5N)w$htUr;R1)tQ&s1GvC& zY2RFRLVI@cI=S|;2YU>%PiST6qEYrJQV{1+O6>!6aHot)2&X%hp?nH3deSLa&*}L= znlc9WYFgpYqx@>0exMYxqU)N%E;6P3%2`xjoZemMr-GHRI4^=C48l|uohqm; z{80(G#I2yVa3bjdcqu$$z~KF}1^+DdhH%t_cAAmM)*&@SSb%>b(dTdx#bB8Cuv~+g z+ZVU>ksAkJy_@}o4@y;|JCX1c;fg&g8X$q z_wWAUKjuE>y1p&Ldt4r@9Vn1g2T#UU*Zzkl;!a5`2zPZmeS=O7(Q(RCvZeNq(e1*4 zNi@`+M$qkeX!%rer}1I^dUPOnDRlAA*SYc$P?Jdh*us+xCTvv$dkQ1A+0kF&G~XNJ}j++ zRRldtj3o>(62hjMWug@hsj7_bLvTSowqQTT1xh9)VP$8ngey+_55qnSx2Y^DJUcTA zOX{TUR4hTw(vHxg@u8#rWmB;ijwk8*`G%09_WxG|nYk_LtXju*oCLD3evcvJtOU!J zHO?`uuApR|FZbX83p4uIK?O}s3L6-U>s2GCLethg(@x@2Z%BtrF`Out!dbmzv=usA zG*}=&V*sJ4cfe*S)e3!iRHR1l6?6QF@4hvAHhD(!1TNtN@BdEW>c>*?cuh(m@_@^i zxbCd~(LdEcarmP6{7GGc>PjcPWqr3O`!47&U=n>`aF zPG5g4oKwI<#uMlwPFkr)daH!tEGsAT;81(50|jw#j)Ueq0?WjFVXSDpTjrbCmVPAi z@HX>}#4#zHlWhPPqR#;Ci(UY}$BbBtDs#@x1#7ee(&(k%=Yi6?nYUm6lg$tcf(uM`L;fx3Yk)j?;9 zp;>zMD7pGx7c$e813cQTnZ#=)7b8FnaB+b>riSumwcg@6>MT3!Pb0o=nr)4w2Zwaj zq`&Qj5=Q35ZL=Oo!@xKhkr;*u$0y@KsMkwrBLImO7QqIy>Umkf(SiW-85^P{-jG1m zjps@mE;P=l9Y;uzGKg6#4Ui!EYoEj*?1~+E>3!VHE~ntK59{Y1zx(6m(=Q#m@mEB` z?Gjf7IrDgM!u4PMri_9=8LhiV{!}VKrK>GXt zkRLw#ZJ~kwE?0edq#n}!+T*jQ8m@}dm{ZEQh@8|s>?&Z?1Y=UFZO*geTmw=04o}u@ z>zss2onnbq00A`)Py$27Du|gGVVBKSX5mPJ-V8!IYh^ZYz^x&r$g~f0i2?mrD0lH~ zvWN1hgiEjLRA%9*hVV~)9>Y$}Qc9UpH_l+lJd0hM6P`mMHsfzwHV(K7v{>khg)~T( zMCWK1@c~VxyM6i~+>;camkvb%1Jn@U{$fVjsRD#z%ta0l=)aH@BPWv=96<6r{&_RP zY|5U-`qT_j%EKW##Q(tfA*rUhR%q=F*{hYwKhKMd;QB>~A;hzoHcJOi2UM7P0w)MB z>M^XrB_lmxS^F%zPmPV*$cSC#2@eUiC2H8v5O{8qCtgtoWrxYHD(}p1DF|oxT7)A& zw}zzaJ+i=+_R`br8dwh042HQn7pEt=9VY@FZ{Y-WT_tJ9l>;PnWI@<-VIO1A$FJbi z3Lx7cJV(9A$$|SZQKqPK&XWFmsW*gtIRaK3Du`MWSi)}Lz-gImIs`}z1f3EbXlwbv z{-6Kq%x-=_GW+-J>!07tomgyPHTZ7$(yS7#h%5o_Fz2XAFdETJ4$7WT?5NGbCI;c8 z+FlL$zJ!N|)Vd_ahv-9I1_7D`Qet=oY-AW-nUh!lguJXt@CT@R_|xOz2_n!?h-*Tv zGC$r+0o#H6!Ld&Fg4jlqiuz7iiU=$5BqMS<<&WGEXdxWBd`#1>*l=g3v&{Stl9g%T z6>K#5ErMmk97C(g=jB!XI2n*ctEFY%s!!|EWSVzk&_+3~_92x02`!3NqrxO7pQ8OcCa;e);+5o8iT9VT;M z3AJ~(|IrW$X*wulnP6PsjO}st%8H~!X^!q95 z9u$cn`G7)=GMgA%TC5Bdp+_!4lL_7F`MrdQj67vlj$q3@;hRV4^rUi30MNmFPd)bi0S{4r4_Fn_}h#$sl895t7 zv@}U7NWUQlameRj>8bLFfhsR30K%VXKC;vsf>|U4oKHw#=Yd642rbCPYeHrbLR%|e zd``Ni<-^iNO;5*Uyif2mm$7&(x(nbOWs2pF3Q=S~;CyN=h5aeuXSGLW(pJ2H29{J6 z@m3JxOm`+)d2*^UTt(+iNm`jH(_dYgPWQ^jPBi}NDdfT-yYGIn37U4kdSh!_74dhe ziOzGTZcx&Iuz7$T2Tmtkp&DwzsGtzKk-69yu~rVu3RTi&{mj6&Dt~OA6{)(blW};A zOy{ABy7VF%_Jx}5W@xc5YV8H9%$hErGZY8oonz)UUYMqO#kC=Rnh6Df`Xux{L+~){ zF3cQ;hv0)Ut{nK>9Zauarv*06%z-C_&83uxf=(DLoE+*aG*X9oksKuMQOl5XZQ)^F zv9$VhIe7y<)JaFJ3dFP+U_A`qHX@w!WN}(0`VM?ieK|q^o}9>*7+19lGsIr9fVek; zxg$$(TcO+O((gzIP;+?8ku#QvPk*fA#?#T4{X+9bByfbp(JatJB`2^yfQ>=l%N9`_yAR@RU}zOq!YAxAUnzQ z^VNZ-z%iE&f=3N;HZ)^Yf!{ok;7x5HeU%fe8Z`=ooG8Q+p366>7fem`Y2<@E+g`80 zBN17!UYtfjSNCtY9M%b#PS|EHxN|spp)MS;UQDE;upFfz(mHuOloU-T+FUY8;?1y; zBJrkYmt+!3<(2H!QwImrlj%Gcj2?JQ?@ao~(b)}4L_@1_qx*5Ph67~~hAjpxiLOx? z6Kj`2wE?h-;j=;pPtVq^FZG6$j)TNo3+B^OAXFauSPn!ir~Qs(?e>pr_pNW=m13a2 zyuYTax|bgxu2qFc+UnU)pjuUjERx!0Hb-Z}sA&+%}?^|VVOlzgXv z2y;A^TDZg$`y3khyftQ9+*-oL<6G@vtdpwv>sI;aVv%vng!u-9>n!D&Ey&Mwr$8fA31>q zhKyE`9lnLvbl_PJH{d}K1_Z{?Gf0OvO5iqOD$|Tmgs>8(;XkcXn|7?OxzV<7Q4qme zIg|zqwq_-BcC{qxJ7ne)Kxm6jQ~@qXRQg8#z|k(mgJSRP0S2n7A#$PRM(zU-2z;40 zrZs`I0U{5OX2l?C=a=d5h;oKJX|B?3a6vVU@sNsAdukSp*=Ay)Jis%qT;68-3PKo( zC)6Z+-4E3MQRQ`d!}Ys>IH0_;cx%}aZ#lUk06a=0tthUnR=mA-Cza?5FWFXh)MV_>6zFUg{Vjo*iG{_Xmja!% z6X1d2tcV1-4A5M1Su@1JpoVZlfo(DC?t5v4ib&=a(8Qi)0e7i4M6Mi=^BVe!2H03; zj047u30tHYUBk8jEGgJl?SFKdD-hvb0v;TScpH_#>EUr?K8_C|F(}NGwSj*(JGAjI zLoA9B0_4Np_>sSvcog`8j)MW56GN|HA3pGl9P;H@x(@-(2=+0=u6USGs;@pW-)Wb3 zVAk?BxAZRa6_|$RZ{}fLX*B|6Ci1O%LSi>p9rvxN|5Oc8*(GfWwvP`ei}e!@I3tLw z78n7l7rbN+lQ>Y`ahae`3Hab8$~RDZr36sY3`4RJ9s(r7M_!&W1$?%1K1;nJ8t;e- z%*TPjRv1&^VIq|3&rELs4dF+m@9@=6al_@uZs~h&FFb$$`<}UR;CxMrKU|XY?}=Xe zOx+-sUh)rr_Gj*q7}A#xvoti!%QN@$(@psem;KwbU)8lDQZ(rl8_7gNx^Ff-IAc2X zZxj`EY&S(*TLE9Bm}10nB<=9xG?E<6DHEqbEw_jJgiMjpMqHb%ID|ED0z_OoO=MWo zB8o44YB@DbBQ29g#3(ONm3)_bB z#lfyzYX(UU7t^5{1vgIA)1X2Lfwin&J76QQT}I$U4pe{US;To(L;#|Z83Jz)EDbW! z5eBI!9SDR*fgh@}of)v1&^pry)tP{?KyFM^)G|P?B9N8Is>%a&82Tgz6w*b=8yJ9Y z?0b8yptnaTfpr+JYS@V*nGUD+6NDSU=&;tJ|AwZtriy{+1dgw+e54?x(mw2zQsq`< zd4Q9%%ZK+Cb!j(CA$94F)Qj$TCxId@)J;4E~l1V<4A2QH(CKGB*VCOIqw z&L|B#Wk+J!E`8i7oh#YsvBD(sY3B=PFmG+Dq}aO8R%^GCR~{WrppsJrMCL9NnDv!% zXNQDWr}piIT;GJG&=?uHpTK`4{J5%e_f=z}6?=>{KEBWuYA6#;1SL83({G8-;~ zNM0_8$11?Md6b5TwqQof=&o%DfgFe;CxNzl?X|vwFlu*n5-7;|D4&wAL93a8sof~a z-^gnz)3e1`V$(|MsWi3jM@seV(XH1Wy^T_(aw~_&J3Dnrmag4A-w|9fo3Yhy3VrNq zM4Lh(d48AO825D(GL$7)zJqzH9}lS?7~+QkAzenHdDyTDyp0% zEF>5Pk00ijP0vKkE!*xf1k_Se;`y4a*(12NGweFyssnG&+#!1=v28)O)PaABLd{{H zeWY8x7-XuZLi_C_e6i$!wSi&@p$9$k(mw7EiTvUuKw(b%*DXTz z9%LY0aUd+5V@@MAq=eA{Q7EP;;3OR`o1MOX-wYm!WSS=FZItLw_n9=b6hD5Cl~hTn zr~j5^`+b+n+xyaw(E~sAQAD*Z;~RXdkGchtw$UOafkg2tXTrjzuCRT(g$S>qm`$`h zsy1>L(Vr+QBc=oUOe!x?ci8>vrXO%S&D$6D{13|Qn zl#x4XP#OZ5YDLZ=T zGe>lQUFkoRB%eu1!8j?HS^+AaCRN8ToI;??CA)(&pnqsuIFd(d9k7hy! zRToDyQp%^s!FkQ$-=c5Oi z0m-0+gDoyBV6d8Z9yN$#?zFaZT9#UpR#i1I+yTkUMtfeT{)UNbI!q z;EBwb8L&=f2hy2S&2)Ym3N?7_r{MD0gMN{g`3TyIFzUEp(iDNW z)=tj=N5Bl&?twvX9b0=~xce1zIa(6?l=}gqb;&+ecgH%?=rFamgz>XCc~CBjax5Gt zRA@b6V1~+Igm(j?UAYNepQ;ny2`;3IHLph+89O_Yc@AUeMV!{if=1$=~rfmkd@6&4M8JW{)AlRsvN&Es_ zpBR4)0|^{y45TDAShSoP6j^Ih_Mgro%wV{J*^ygmlVwbRM6LJ_DGM zO*IG-wy%joZbO7|5P=IKPmlV~FmW$e$z>4>s}uJrrmh#POiTlhfIb%0Pzhsxx&wOTA%;Ek)}%G8|8` z?Z#W-Xv-flQ05TR&cGTYr>2Jcu)qr+p0%BM5I=t5Qw{|YzKHRmGZ)?DPZvx}1JNm7olvZtmTSTZ63M#ZdI3nYt=8@ua<{86PlFf*xBp@ez zkypblB+tyKpm<7YdNMgNMJR+986NZ-&|;>unoHwD?a6Q;j7;i8*!ts0PQ?Npns=X+ zmQnJAoh7uYgQRXyiD=tKnLsZO3xw2YC`D(L3j{ul5)>HqdX%?sst!16Xu42o`T@$9 zx7y}@#3X=S8H5twYT>pzhgzn{>@Ddqu}E)QvJ=6D=ry-J70h_b>rZVcL6g1N6ArGqzISibWSv6N;fShi)#F z0%MDiOhRn$GO{uBb!&N-XyX>)lf43_&U*iuo1xsMQZ7!5VT>jZS! zRk}JB9@!QkM?6b9Z5%ZS|5h=5FMJx5P($3KxUMp!o02`owhrl=Z zMM#Y@k4H|#U~MSl6Y1F^yq0=Hf^eF6Ytn~qChz=h5lF!+!h((}Q|#3)kq%r^yF`KL zbNT2C-2COweUsbo>1Y0rUt0fSLmNkCC%~-Q=P1(vH=LwtGQf$efpU_>OmTY1H$&Vl zlySm-s$+Cbm@2lK6J|Icwe^h%xp6-d3%)u%P__5=S1N51b#Sfe8{sOV$T7JVAI0D! zf>VWzlrDl*;RMkuQfek&;6#(`4HzU?-Vw&OAj!tMmUM#nN6qX_$fV^@Hp+8{rC84xeM*3jaK;Gw7}^2{P+K~Z`9^6G&#zAgYekp9 zQO?`ztYm}87zwR}{M_wVvU%c_eKZMK+6iJVpNI@xFJ8r?^`O^(@Lbv3mFNcLphW=> z(_EwA2MzHk0wf=XF#sQsLlEfAFHLbU?evUhS%9%R5FKsC_Q6<&F|Q=!(%m7`F?#^pG<6?RWB~~Upi`b9Sp~uDF*A_Iu?MOL zMr+|5bZK)4-Zr7V>(K=-L*F^mWOb=G6tE|Xn8B%M_QGw)+hVu?pX9}0Bx-6H2JItp z%%2e|uCG7#2)DKT7?0kl%cdgHfH&3w7hAkMQP%9#JMX1;*qF*c>ydF8aH9r=15&Tt zJ*J2%($Px#mVZHy^K3ne52tcw_~#dd(zp2v8|Q4#4IR4P%0pV6vrLt_5Jq(A%7xIK zp@WgNw(8=PJzyc&QigfyoATWj9E~uK%Pfvz@kl$4W^bw|=S@EY><`Vgs>SvajvNN_ ztmcA{eziL4c{pb4P}NL&8gXuGpO3hHw5Tt`xuDr#F{p{CM~|pILy-fF3zkjnN99tF z_A5M5Nk_C(N=-AyA(cNgf|vuXmSGwVbPw&yQ9_cxn!izUEc5Vkl@^N9FO8T5ll?rMsz+Q%p=L6KTHBG+;^iauZjxh z&ZM)GYD;9Ew!@ispn5p|E+C@hm`yb1KGS7pKRyl7Dr z)1M`Vbm){(Zk&;l0Kys3HV)TN%%mm@og2vrC5S)<=^3LE$SyQ5RgObCTi(f~-jJyi zsLIxYaF!-|wCsW{-xUR9d@9^1vow)$rfXmN?xknEuj79{{mI)><1{bd(Jl#$9IUzR zk|hvUIYLVMntP*OnKQX^km7()A>TE<%?uyWZ0W}1z~pjxOBwfe@bRSp zdF#4?d22Ggv!_&e_#6jh%1W2lu&zScH;@a`Zua-nc0zr~fyAO9>jkAL!ye(S^c-E=-DT|9lMAH6tx;)bv(*UdfY>b)uqD0e?8 zox2-1Y9Gjyhxou5n4=(k%?YFFA4{cl0+4MV1tZbeXnBKl@Aj0AyC`k6i=!HO@f<9j z)0ztGhz-pPx8d|u2e5t8;DOK@4pwg%CiLGTsHNM^Fv6e$3$%`{W?7!lcoP18VFVI* zZ{{<#1XZnYn}^I%cTV7bUHLVRpfsB1NH$}4fb*D?VnxK@m0c-B?87oL<7w1|2xZN$ zLC37@LVMY0n=j-8Vn^C5H7WW=VMzC6l+&3*g@8DZYVigPK$H)oknx#d-j-`X3SWb1 z%Wp%^s*x*qq=Rc1E09zO$MEw0a@NX(n@G-q`GEAaqZIMN^kCdSpx{Qh^M%FGf##(* z5)>iyt!RZ)UKhemx=@U~#Yl@g3}_a|8mjpU9G-#Exk?hur8H!6>aKGB~6NRyY+YxrLw-6Xu1v2-zchLDl76b_-jT& zVHQ-dvrsf;=I`P4*4+VT=^{R)DRILyzqe!A2f!eEAA z=^E76y0fX{1KOGiS?i|fGGo`Ith7Y@wdVxA^D|HGNnmiD>ps_yJR!iFkBJ|@u~zNW zY6ZenasI|cILWxYtnc)#V0@$<`tbDwZvW&5jXtE*VR=ni5aM%Y;X zOw)d66lLT&)6J#D@m|YZQ_0`-?VT(F4PBmDwjFUEVg~VT=k|`(aX0Ef&oU%>3qBC_ zbr5!?dpC2?!mK^O^o1!MdCEW-ka=!~wHBg82?M|`Q78ySLA6Lq#?6qDoL`=rQL?o< z4H+A&MnQTZvkRq*Nv;!ZQm8E2@jP_px!CJ6n-fTad`AQ<(xxIDZT|nJN0v06AE#eHl-YRZ~@Ne4nv!QA84EuOiSbTtTJ!?ye*+uCW@!bvv zRZ`b$c1Or-xG@_+g+~C z6>h#NA;V|6UX&`siNJulPzrcRVg`K?1vq^vxLfD?JXZ-ue)4T@W|tSopPW7B!F%KN z=j#78yj8~(Z0(qs;^0_2u7&`MViT+jnk~I3GhH3X59+*VhCPDs^!TVja2RC?TEJQ% z-V8aEPG@D!?aNfA?O z01S{(7A=@=fKC^BV&tx4h}5%=m6Iv#c{8V~D!lh0+A9#y=p=1CfoStchhY4!28{rd z^X#}zZ&lTtS8+)@>(|~qrq6D=RN9*~H`C?|X-D={H6c46KCXkk&2kiF#BoF*l|Va7 z$Q5Ca7PhfGq_3F)N$TW4e9PS;ydGJUl~dl|FmE8c=+(=*^@3|q#ahiFFxy_trtZv@ zdJ#r4BBS=I&vUo2hIT2#&+}+#=OkRn9DXv zIgom)Tug)f72?~BHKTfhD5re)y;}`Rkkx=0oIM6>&T$|Gq-reNWvCpA@zHKSS7ewh z)^OxhO1|B{*N`(PsIUvot;eYp8UR#g*mbGLuhg(%7w9(aPPo-tXlkMK=ciH#1l3rw z5R&Q9AZ}8hnq{}y7p*|Im-!^SbLl_9cFjf`h(X-s1kNyR)A!NXdcA;p`dj z(I0i*vNl))X<`dE;bOOYwXni^CJb+1qj+ z%&GMpC2V;L#Rk>Gz?2-0Go>#pQeqGl$*+qk>9Z!vyNyzbxDqL42;WW%O!m~_?p`_T zJ~xrY5Z^5p9tbpVVT~>r#Q-!$F>EfppqN3uIu0V6^sG#orSJ&}1c+iZRNk=lW6R=# zq8BDC=`sFR{=^34(!?6K@d8RE+c=g<1boVF$><(|cH~c8_EvpaU-VzB|rJV); zTr-%Yt1)C%?V>dBZ(G)>z2$1_bq-7ctZH?uUu`PwFctNjR@1OW8cSI)HF?@jp$3?O zBSsf#X#1hGU!hPSKtl<6W5m4CMgp0Rpc*|!#?RJ*E%SyXcMxH(1!^XAX2F6jFW3Tc z4v8@QDvgjK2CPwgeYwU4U3>4wyW{VFAUPLGlUB^xXmix4 zGF45+vqjQWLtqHQF-$;jWd9i{&vlVaJeV<_HHK+G?l1(+GHjQi0wxdTNX*$KHvwWr z)6`~`Tw*D*C31~06h%-#t^i#bt{`g9d=!8}zmxSeB*|uErKRXFC+HG^8_~s`N(97P zhs@{m48#hk9Iggcq+)v-pNkTn8mHJFgpR@lu~a%^S&GpK@P8X4%F<9^j)2&Zcvg8L-OhGH zHVk)NBplbB4a3P1dJb?B1GelJH)zlIJB?WTe8 zQK4$jvXr_@DSSz3#Eh*&f*3OYhbO1J)h*^XA~N% zjN&x?-qrG)7b!|krfBvG5W-wI9k8DlT`B8E1CO(?WJtE7ONJVWvk(AY*{)izuiJ$Si#s06$aTx6~UZ0Lsvw;37jO)RV#VY&&Q` zBwH?DP{L?z*!VuGho8a$xz9!S{^@UPwp88{C0gi3V{ho?3L_2i}k)p)#@kJ27_KQPSku(N|2QZ$X`V{kAEzzSGMYmLtj zvM};JftL&MOC`L4DF49_iXxz3XNoSc$fi-77KC>c=LK=4Bg#X%W_l$gk1?zim7}4< z20Uc7`jt+9fk5icQ{8U7Li1MXQIrS$fEvb=meCi*WLjKag1#l%>fQ#ZK3PM!Qc;I0 zHr&);BgYehkwG*xy}ZvlogvRWAaT$&Mu?9Ig1Wuw*$ zrUP-n;mTgM%T@tA#7i(I41@;H)BnLj>IB$a zDQ2Z)fJlArj&=_MUc=zz6&V8|jGX{GLhf5v$Lf{3c2uI;18+w!NiSVhxc}KRi#tC3 zaRMqvrB81cjY(020^3G|5t$0QI>15z0~Tdx4w@!$A5^PKb^m`(-#2uyrb9KV6_er1 zqFUO!sp`O!NTJ8HA(gXmSK;U=%UVug&d4gQYoacno1m&-u5wBkax#clVfJJ{bX3_u zUv)7Ncj{sHOliRywhD0;TbG}>fXj*z)v=IacgZH|da)$|hDTwE9VJ7fb3H`HNf130 zU)q%L)H71T8@X#$;HTIyCdYiKCzP~!AhH7!A7pVl${Z(*&Xz}EsW(K!2mNK{L^arm zZSSc07=Ozdp@os~x6GJ@hyifv`qJVRdQ}7rKQ#N@tIt2iW65o~)c&8;T1gga>1m{p z%Y%krjR}o(X@tJS0LA?4^D`gYp(NQ$Bw?9<|emR*O*C3 z5Oq-})#%+37cr;`RLJ*jd2F0j=<9(Hg4b2Kcj15({wNq~Kr-T0($4gl?!t*bFmerT zRg41>$I)hDHyuP(t3}FM08Jpnv;+-rq`ND26bExCOZ4gN<1%>;ajoEkz;I$@1L*I_ z!+<7`=eww@N6?kY$7v;F`;8E+u|mzX8y!~r z=6C`xL#klu0C)kh;bMnmH?zd#*bx|3JWt3INF3lGLjzgDM^Xjm=(?Rcp?R;z{mlI| zt?RJu_~1yG zg)-*H13%t?_LVj!3N-ZGU^W>Q%&aleMlyG{_~519kcJHnoHYu-jm!4DZDbq0Xv?9# z89@5bfPO@Tbgi8E<vPwBLI{!m_g~_s#s^P%_;$oi zbgxvU<<%d*u~wJA`Qg)CD^X4sw9rp{|6|-9r*b2B5!keE+^KxJOShovgr^(-k2fQZr<;eg6N)x=&yBG!Zh zQl)J|g!ai2v3kXqZVd@&`(p)|5}s%Gqp`h-ehhUtYfJztg~Tgl)}GN1dK^s5kif$7 z4oh@F|29cG9`ER6U-cNxQ+H<ygr^feSn~u%RJId4rZ`%iwNwiS)acSW z5mOY~{qwX=0vvO}5fmW1HMd|U4h@ge*@zvQ++gw1tw(x`51!!*;k*x`jvb#h9$67{ z$!=+ievssvmZ}8lAPga=Ap%{!VNB18)A(0UV8D2ViU?&U!@4VzrIqn?SEjxH5@Zz@ zm$(OJu_)U5C>aqcltSO0Ny3ka#Rixa;7=~pxLq-0G_TYrG-9e64kg?<$MjB&bt3*J z^p$n%t%ooOzgn2^G@~%IXBd|04vxNppph-6g2#55)YV=PDL|xWKqO?eT`JrFu#=Ps zJ)%Dhgn~v>mpxcxP3HF2c#`kIc-EGCW<&|1mw{zkv9VNhU|dNLZne>`7x+w;D1kPb zu1yW~03=(OPg?Ch5Zc+bDv1czJ>shZraaV(_V&PSBx>kX12jX))N_x zgR=2GgbTG<>7rABqu9`KPt@;Y2%xQ<*PQspC+^$G3s|E97l1flYrrH_8RTy0=T6nZ zO{%G|x#Pg!&_b}`P>0w;lT}4To)P+N+J|Tk9UivN!Zik5$2QU&2L?Kw84n%@d(iB2 zESPOgHpn6{7A5ond~19}2HnBro-$7z)gzVI;BlM5EpZU6wv5;j z1wn|hTAnY%bjOFaOENr?OmWfmY%vkKoOA=1k|Ex$~NeEk)GUhn@`2xmJX6n#( zsx==NmL^y*Lfgyf$xPj$u^INDHX%Jo)HiNptll7Y#Xx!;S~B4bMbSC%twV7UTs|dW zVLIu#^cBd1Ge#&;*QsJ;0i81PJ7%K^Ud;#%U@_{vP3tOys)$#$0uM^&1Nbq%uhpO$ z=cZH02-dGOH4+Vms&gfRdOQ0{(#w6WNfB!gJ~7(Nid6e|~+G{MVcdx-Byt zbqIUDW}#63pVXLTAk$bjALct7GkOjFTB$U64m6^ay)NmDqv?g>{H-j_v52WJ%#>6) zry)UcU4a|0CTI^qw*ciMaV%(0Ty`34(1r<%#3xzG)X1=E|mV#z;vZ zlno^^fQV%~Sy@uDPHWpI6eRu!87q+SoR zVOqwnFdD>Q8X^W8N2nz;P%ugpKPAaU(^uJ6{qH9ZT@E;-`M!~Dq*X%u8;J@umf=MR zgs_iMCgb(YU7#8lLe~-?jz;7OC?z5{ESPSm-7ysc%q3$vt=DIu4jJ`ER0ug@wzCgX2w-dwGyLuOY|8Q!7{U6u6=b$#TN@0#vi^*} z()fblEORo3blfzopqw#2e~F@OBt+RE9ctDe2Z(|>btEvUXfTG5rnT0;hL(6MKuh>u z)*zvTOoG}c#N2$%euPPm(Vs%56~twbJH;83+${BmFx)^*x(3?1SrKQnjZTt-R8q8w zqv7b!c-MF+gXQ`nuJzJ4J#HVnCw`Z^fBh@^7jJU+f%`xI&FimjT)juiP+Y#m^+S83 z*W`C^g?su*Zoho|O>W0h!uq`a1j3*k;)t1nRy^CV)9B&=v$S`PRJU}B(1|LuP{Ky# zoZgeDb8Vp}p+fr^k(-Y34F}!05z%wrD;tbfd)SdT5br3)AjmDGXHlmWRX>(+C)vF8ceq-nO1c{05F17c!#)6lE zogS~RC;!^Z#DsY*)-aS5sm6A{%Lb>-yq?%dS2|PR$1vd_2C@Bo10|HTZU?U`Ses5! zlYUTZ1C6|BNR*h^e8oBwF~NyyAeiO2>(jlezj{I{und2-OxMkvvM+`j?VZKK_MiK} z$1z0v@B0k50-_UZC1hH&%Xjpn8waj_@aO}xUrDh-jGf5x2pI>LkgGJp-{8^bxxU`|1y9r+iHkgZh^u@MeR=k+djyAgfosKee0XI!dEci4{r_`Ye+4|+ zmF0e)d;=4%Twd3A+CR*FfflR2^7{IZ6!4_X32=G-;VV=A_a$U0d9k1`B_)0PL&E$fA5UNIAw=N+G)Uy`qThwJNK`+9mD$-j1OUpBP9Jo)eb&fnwF zD|c(YJ~x;TWd`{A0IjbT=>Y>4*8)S3}t$bZGVau&$xdAX zZM!EQ{fS|%Eqd~!5h2t3cs+c)Oqzhbo)o!j)OYhR_JfCuNY~br5kh&I0@@qpA}R8;(N;{#21daJ}iDm z9*?;B&jyEM&Z=kDcnsbwSHB9nH%e^6NYUq)lmed7lx9vGw0gXBMXQ&8Jfl*yKy2{~N z^yxaFWskq_+~>qiO7j12eozM{Tl~P~(L1l*FQ}6HUs68qM54mSU(vtQw_nhA@dwWH zKtj4ZTwe<0KqEc)pK|+2uJ@`F$^$N9CfaAQ zNhcc0GTD^VwxRMRdB$0$lAGum9^q$)7WHB>d4UDX`b7jNyQ z!ag8Ar@hlCsrU=Z{JZD*= zsuxG0*OfIt)WXkZ$^W!*&igBYPQzbfpg}IlAJvO90OqQ363dperTw-3oRG+w`JbZ0 z*k@3syL?Fqar=U*Z0#-kQ{%i$!3dWkehpwu1~{m@qP|NlF$H44ym;8@jO_Kz5HME< zr%|SY5fKSfAGTIRHCBusEwQI{#qfcvS+^HXCi$*_T%|9gLK@v7QCsn8cmxY=^xC@) z+)gSoq?ypQkd z*6Zw~8LJgD)v9=)t@~@WxgOySUVnCa&>}Fsm`-f?6?|x@BR=~&5wl2rp?`ZtRF30_ zxdGS+1d0g#M^?g1S$X`IkCKROCf}MFzV55AUR5;BrP7~3Uc7rutwcdcy>*9~ik2D^y1V^rMpq%atE*xh{<_N|v*{2EE4h2irNqAw#~( z-A)k%mIqm?v-T<#kyiB#f>A1e-xIOcQu~@?@UmElSL^^f`d~XDaIdYWd-m}*ph`TtVTW)N*k1GC9g!o zCUe0E0O@cdio0G$pM`(aj8k^T2OUQG!=qYT(@^{o5-w3eXd#p#@3F?gst~kd=5Q^g z`KtgOX6B&cl&ivWtUhTw2xhRoMk5UdcLKfb5AI+XPjaE}cXL4s(1V2K4bDNgBqFZkaIF}`iQclzFkPM=O%K{oOr0d`)#%3g)d6U>AhRMx zuuld6v;?RpQY<2HkCHI8#}P6I6}^G1fanGq%lR#(i$Ux&b9oy6MI+Fqa=Gc*$18!W zuGV+lPEYpWA-M>(O#^PLF+rC^IisBP6E4EEku1tS2iJ%yP%Z`)Ybk3akk!=7UF6$f zbcw;6f%2VSBTITPpPDnGJfxnxGFQtpFm=bB7@P){TJpZnM zoKl(P^h2;wvAhg&D+h^Qrh$Oa^VtBvA?4vwl8j)8Nh7dC0WZ^#AV^-7mxtqc(1(z45h@}DlB_J(Sm`4y!K~l+~xCyKZ#b4OS zG-A0(W89Z&;OvrqSXyT9eCIRu|0-3*XXIsHe?Ix=fBRP5Zsq4~u3V(2{NTsr=~rf+ zpEZJat?^Dg&_Ihu3lxWaEMk6A^1F@H!XB<$3DAfw5`Ztb2c<=fMuy?OuQE|d3kkQa zjC$j~uQQP&AwUm2m4TQfkRxsN;ozM%(Srf5_ zU6{WtBe(qq7owhIaouVl0V;sBJC%U@pltvSg9b)gF@ps{rhv%-*kxBQc1Lp`m@Wyl zn2OhKQ|mRC2r8Q)A`E0YSMy63%gF?V768zDb5wN-YSmP++nyV-rO~3He?E!CHLd0a zBi%6q;D*7{&)oFGn~+HWd-fRKWTGJv*eF=d48zKN5$#(y#^ypg)wNCxT+6%75$S53TYTl=|PB0PpptcU;EL{@Y*wyZYe;?jpg(eCm1q#P9tg_YZtV{~(AHE~Nk}F8%PA z|M)k!KLu>?!}q>;p*BNmF~kf(uBU7OBpG|``pSFB^NWt$5ODl&|eu3v5G zUe7*Z`2of3k5fXr5nj$G@_9qE$mbuIgN5HK`>l`L+0V^W6jNSmbGYy)e!!d%M z*BFAUxfMEqF-4+N`FrJen)=>|id{Ooo_*>QvC!45kES6|jD5A3Rvb}TfH6G)A|HefZ1-7#S`ptu?huMW!PnVRLIE+iR(t846AcPZ zAr?i+SaV_BPEY927)z!^0xWxsW;0Vj`paK-k6nPy{&4+5DIFp+5mktftF{3yV3~R= z|1}d)F!%+LF^!l|ddozD3}L(YeS%v^N1O<}|NN?1Gy*b|r_2Q!VJ!!!od_ydTOd8e zCOsIlTlK>DFl~CK?Hv_jz7{z%c9cC0Wx&D%XOC6Nz`QVCK$1>W%%f^lF+2dfql(AO z%0@cVX)-O;tg8p$T(s`hsn|j6#MnoMG^}mbiVe}6vE_qu{X#0*Ip&T6J_JRb zmH8+35KCQXx4PPU00|JBXaxc4nVW79+>F->I236fvf@4(+Gs_PCzZ?=$A0BT9z6HR zjP^tUY!le#E2v#7@*5Zfq#!_QgA$obo;QNR%_|bL{fGiUpEOBI91ny6OMR4)hs6li zgcPKv%5(_>$WTUGutPz77I=|{HuzM|Bnp_j2y$=d#@gfW6GRV}3xX=*BGA9*xTb>F z6JRix;N>sfUweA>2jl6<*5u$ZCC4_wwTqxU#XfmXaX~0bdLyC0 zO2v_tRGdVmvK;4IVpp7~O0g?(rQ#}8RIx0}iEOc4QdCmKNu?^~{1N#*Ywdk{?X&kj z7hB3F0_WCedZu@;v(|fk-sb_9!nE6rHLr2+->%*QU8%p=J|M8qTPpKG_r67wegxgW zp-^r)crRdBkBOK;4?jsKG;%YBgYaHt&qQbxwn!i|4Hgj&;=|?*%Wd4l8NH=@fJ`A! zFhcjl5H>C#BLuZsvk9+if$JIk^CyQ)Cdw;Vw`GGSg9YlYAuG5FRgbMzIPo!vMxdzf zVX^Priqc!)P9YLua+bC$v=tl{ih}4);oERJoEJ(vXwCN=77mxU0fix-EcKuukPvnr zfJ*>oOZ5yvun5(vin*=PE!96vLZ6IVFrc}Fex)mk8FP#%B{O)4+(kio>}@}py{ylS`MqyEG8^Z? zLxI^O;nu;Ck0&Lk7^jcVi0YD$i?zxPjm_L+BP}qWlrvKZP~s52tKFV+1QO0)jADiS zn6+kjVjeH2FqOHeXsEc3o-ng}jPwu{Yy5&)z)S@$X2~X_AoCcaGe6mNL3f?$NcRji zj5v3o19AKgqT0+G$vE@^an`1CI(PRJPOz4bzJ!ADVI{y^pDHdRc1sm|YNSLf=^U(y zdE@XFwwVb=YQ+ho%Hx~~A0Z84N6EG(BZf`UD1_Z&Jb#94|9~?z2K7X(8SgYAEE<#c zGhN3SLTHz{5oj8iQpDkCe#jgmGy@hmoRuymfNN8FFwV#&hsYB*GqW;dL8X)W5D8kR zL-fk%`r|x(f%|ubg!|wBs$i=1kcEWU9^)Qe7fjXOo;fUPf zoPA4xb%A4l!G5VD*DTYGM>u9UB|s&%wtFOYJ`+-#j+c(#wO2TOyZh$5nhH5}c@=H| zk!xy4$8yGcY)fVM0rzm4ZiAIqG2dM4#4jk8(LP|RNk)8V%#KeL+WR04)C_3(HqHZB ztI;5Y6&)3FRjBuUHWHBxm5r=GD3Ga4W>l#ijPg!imKA^mYgSz+b*U=5GWrYo}%h7>9|90-6BJ*`3sBd#68nrF|+SY@mr{P9GAw%-S;WT~!pu z@k!L!JMg6=*gfiv0#Fw{oCV_yE^6o`>jDX{RD@ipsO*-tWSorxp3_ck z2*ulR6oJqR%p*NjvjBH)PkUX|_su)Ee4nbmj}c4sQS^{6(f4_ZacC9Sh4ZyiZWD<*j z%c?9aa&ls*7-UN_G1i-4d`~gBN^GhTm_QwVE5ZXJ)SXADRl3$o@X}>n8NiJ87%7qo|056}K zQU@J~CnWi?Ebqs5V@BtS*uI>B*$>TeH1FIR!{)q^Q&7r$@)@t9yJi|B=|ltwrC<3t zgF@T^ogy|7Jkfj!R~zA)zg0~*!;}y5t(4)UEe2^RoJTRFIEu)rghFAo#pvp`|}&CW}G+8*YO z`mC7W=>{`mR$4iO9IY|idkCm$|46KWC=>U83U~u9=L!Db#NVie8o79cifES68+(Uu zYGqnRM5i-G)=b>@kF5BfH(GpX&{>pI_vb@!FG)bdQ~&ga&Pev7J!3o2&lb<^2~NuQ}ON{W~k(D1NwpqUI~tOmy?ooXT>VAW$^ z1a@(}JS;BBFlqy%vfgyvxzlVf+q`CU%-vky1e#Ab7_Ii|AY!=!J>&&w^3Q4KDFnr4j6;&cyRw59X~i?tVrlEVs!C^LK5A1_iSDygf2wum!M83QQl4M{_$DDp7| zOJfHtgu6J?qM*Muw>r8;y_fIVX3ae{A)KA$g=9~|Lt-HP)+|daFx=a`!Z4bL0!wuyj8qlPt09__=Si)8}8W*0q7I(3v z>tRwYcPkNCn{R|Q3C1aRnUDxxv`qMu^_EFkvR>u1b2Ksc21@uiWEi2A8YHUKZC7q< zSgtF#?-1|aVz4{Tf-zkoc*7y9WBC}6<47XagkA>L9A2NQC$JU+c*;+ z&BM3<$$x&;zUJ}u&+?$QwY`1uVS8zIiEccSn~j#im(pZzGb5kf797FG1`UNhQS@|b z%$(usp#-3bRz;IeaByw+F8|z&*SQIzFR*zKlQ6rueac4R7%ZQx!%ztz2FeR@sG$dQ zD*#|&(I^CQI1~~fUP#Fk`|U9%M|l<(2N4QY5jF=+h{-66I~4$6tvzw{UCh(g8Hrbl_@|Ov6g9x+AAlbsudz=;`jmkL|9aK0i*XgH9BWGW<5uZ9ATDM14|ib6{B1C^s@HoS_5-L#pE{kZl|0L6;Vzms z7OB-L9!Hz3>sjF}iDP!%A2N(fQ3m4y6Q*@_I z0H;7FvgGqU03~9KX+{cW3zjO~5>vf)wsg{{@dqsw@n-$0uX^$tpm`VlfIt|+SN&%X z!H!ih)LpciN4>6HcQfx08pOB44ZtFl`f=%{sQeIDP}z<0WictM#77Qdoafz4Un3&X z2HTP(fjNj$T7O{zq(2RGqyf>=c7xX*MG2w<=nd<>!8mHA`fW7rN|!?Wh+bl?XWZb` zMlrDgl{flM@BuVvTJUNHO*^UGwYg>_8oBHP+a0@8O)1Em64zVPT{X@OWWk2i9IHKM zZh4dGV5*~zceYJZBNxzQ`i+NPpfeoP{W4Mgpo=E&BM1z~^E$(tO9_y~O$W6X)5!#6 ze2gV6d}3r&uaJo$EK4&e?gQZ!yH)zh?xF1xx^@FdRVx)mJ#@k5Cj_b~l~>*TS1D^l z+ zqn?7KanSgJVW_zZhdsE>b_R?u(LYcl6hFo&F|*MmIGi$0Krs>6Yvz@4-dRJFlwwca z^)mtq1{pN(si1`?AeH&;ZP5~j+Hspzcii~Qplg#Z6{Xcb!L$bXK54cQX8cTlO1Dn`A`gbgky9sBX@9{Abx7u ztQ&*-3z>ctMB0c-<*|XFA``KP~zqgcA2LXl$ZqVK`O1d=IT|D`wQI3ipiU$Fe2Ch_J zhB%NeN->$B#OI|QrCTEYnzik?x~&Lj2ZWu~e>pnIU#~lTU>6V%rEKUdAV(VN{6X6* z3UMQk-FYhhum_86_|tFACRcvwymnzo&# z=CFb|IUUp(1QW0nOs7o?Zq3j+_VF2cbtG`|(pV=WyrqlBtcT}5OBFW&h0cj|H> zzNx0Jv>xm8|Khiv;ME6i)m?X9i_}YKqw{;eAZo35KElH(cM(&1&#IUx564Y&dXI4b z-M>*Yxh6OFpQs!3+iL%Fs~w?PryDe)s07-H`Fn>r51kG`Gfh#b&=3_)a&kh9E|8x^ zZc-4G$0W#kYj?)o#w?8dXpc@$?Gv2s)VR`XOi?lUVJ$`bR8k=4R4#I+95=c$BaO52 zZBBvFlW;r~_e15y3B>OC2#_p`VHSbi9E#SWd8?V#;Ea_$hlsB(UEfqLU@2;Zv1+g+PKMrm+`8G38M9LoU4roXm< zLA(`gL}^9uiLDfkne_~%{g9>}<8&p{&MBI_j5_oitz@GLeP)Hx@-3LV!{9+&FrGjP zUJ~?1QVc^j(HO1J)HYh zB8&7g_uu~&As`>-{+ZK#4dX?Rp7C|w(-_`cA1vy&#&>Ht5pOC;zyHKJK_A{%TPj{& zerStFZlAGFJihsXKlqwI7HY7vMnBfp-xnLMs)J39DxSpBU z7R#zV0z})eac5}M9LqXz zt$Ah87uSc`_J=_KsM9aqS;LE!2gSrs^9jS^WJNTmCuHH!gMgoau>w4N(Vc zs3*W}kaa0zY}8El_VvI4vLg+*bVqf=h(Ty_e2W5=9|P@%^N!$WmFZ_FHD!r#_OES<;~=i}H;4Te8T7DasVZ{zBr`<;%|FnW z%>b#A6EesXAZ8Yp9S8(Iywj51t4wwPkj8)`?okMAxp_!o&_NhnEWje`)mtoB1e7jG zjM-Aw%$TxOf2zW$_zHAQ1~uAh3=GZ-`(eVoa2lawHWKTX0O}A?(-WvNNT{I*tQdb_ zdaDO;Es}Sd3PlJ57Oe#4w8#r5Gyu_gtF{;%9#2M82t$ZSurZJ&)os{h(fHO%mqHr} zP9xutG`Tj0-oR*^zAuw#O=AFM0&6mOeBvFMk5Z!x_V&P}^2Y2Bcp;UK zKln`^UQ|G7=tD8&0;$|7=v;BQX z%&;*(3iMu`_7V5egyBf?PY&82PKqWIytq>eK`?+iZaCP+g_5fT?FdR1Wpm9D3qD&> z9fF@}VAvV#jfo8oY7&%t!Bj+JxDLtL#zr#Xg)HuEAD4Df&J86M=}tNr`n7qZQ( zofBn?<(wK7Tq~n}E(0tx-qlHEv&UTHBbfwCH zA!bPE+$Cyq7ResG%Q(26atWZ$!2X=!%~WR;EoU~2IM!U9uWkW!NR09M^!X&6N+eK1 zcne?@0^Dwt0eDHsy3p*ap{Uy*Z8P-+CE0w`jn4z5KggZorP%}kGMNMf$kIuMmXGG9U=BH1 z6I&tZ%+AWun~QNQQZa?UR&Soi6q%wluP5pfv;a*IE^Zr!kAA}h1HgJG4-h-q-LpdR_@Ic z^_m>xBu23X2|a92>tT(rF~4}Y`SjZm#LgO2HqyQTlPq0`iJS|tY9ESC5w z12;5JnDzImOlF^EK( zqmFyK@}1f{nl+@QZX++YBKcpQ<+i)d#40D+RAO+PC|@6BVxWSCo=Q|bf&rcN0q>9? z0P(R6u9CdPcHClR9oxp1Oj zO-I^L4sD);_QLcF>L$WfrdN;pT_{c1r7o037s)>bC%73^IMk9ymJfoIqb{^wu!no4B=I&VHMA33IoyrQE;UN zSe_Dh(v9)Wv;>X$Qga2!J5ef>S1#$?L^SCpL4kEE^B5j5Vtjb+>WWp&SA(`~rAuLO zfLs@f-Ii@jc7jxMh16tbF+Spbqm9>%7~e+~yMOjw*TB3_?Q(8s{qx7)Rj=LNj(@~+ z{K+34{F7hi#SeaX^2fh=>AuDXYrAE?>zkW9Pya~pp!;Y4^yKAzRpNjAtIyf*Q5F9A zAH4njcdb3M3wJ|ZV{1+Da{91sU_yk89n;?1vD$O|$qBMte>OTVlIS|j)RV>UrS>3+h?jXFID7sYHyZGa2XqyrFkKkejBK#uKz zi@S6+W`otbjS!2oNJJ=u@P(*Qd*29St$(Li_$c5qp1 zePJd5MPtr*Q@q8zGnj3vXpG#8k&}6HhK6D`n{j`xOU%{ioZDp9#PjA6C(xo1j%WcL zs?DX%hAHM0ir!RtaR1h#PqAKB(7+KGnt=;JS7g_Xu~z1bX^8fZYf>axp}a#b_fG%z zVo^$jB{W6^%|!^W#TMkSY9r9p9Al3mEzs4tYi8=qcK)oJrEYIbP8Vm`B4h#7xaJBD`-{0@&!(lKtOG;FvRyGk{pE@ zWROLNwU9bHh6lm=JQxQ|62PG$!f-*7q5NbT6BFVUppN(l2qMHo8Z{?KD?# z$`|Tkduuf>IdFsalF?mR%A&on5gryn7=?IX4bBn^0~gF7-`L7vP|iW`QqAnpCo!7b zb;t)*J$+zk0*}S#>bMs~8&G}cqwTc`^r6U$n!677EmRMjCu&)E& zC4&mAjL}kqNslf;PLUr%0MxIL&RMVp|;CH?EbR~I!WUwm*{%imW<*L z{hDPC+q6Dg1zP5YVqp*1Pr_q-pc+SQX5#e3u8+ z1WsGEOH)PQ-J2Uf%cHo8pX9#FU9rK{5f2~#P#qqaH3HgFkHi}gl)?jeU?+qG@q=0CRp34gj5~r&G*>`lFc^g?LOTf35h~q);Xncb zq^H|8w{Cd_nL{W0syPPv-faJrd6YJkg%NlL#reY_cev(I_I1E(fWKeiO?{`*r!dZO}s^%H;;Vn}H3^NWt8~j8vKB-h`4XLuKmv(TLwl3s{)P2^6sCLwlIz z9(%6Q_YDAx2tG~#tK{H`LS#3#a!_hplSwq8!V7Ydjx_e1=&nw|qD8>Nl`e&l08lB3 zqifGmQ}pOki6Dz{>!?Y=#@>d8nm+xN-}@hV;`ZL}@NoIkdyaT|;59SsR0(}yZ?fr+ zJ#1e;u9EKe9+G{(xqqF<@%0BE)|?f$pSDl4$AA2v37=S3av%Jux+8U}<`_49dGS6E zfAV)5c{8(kH_mVhGV(N_G>$EkzF)n4>=tXaE$RZ0HcMmB6&)O%?!84Ls%I31d9=ht z&x017I2%$KAMsiKQOBf%3EF1f#7_q453+nxF-4t&*pSwM2ZTKb3q>Hn`y+TPgEE*u zc=;Su1KQgU1b~s`YATwc>F{8SvcU@f{|x`kN+biYzP45~gz= z2W&d1?kZv~lez?EiA07_N8BAHB?MyjQCz~B&F|>ibf~=cPHYRub$l=rQBiajDD)w# z99SS5X0BJae!kkHqt$~Q)5g01s&jZa47#O%tN1)c`*;#S1((5ILi-H>Q0AkXQ$efi z=aX@TD((C-nduXj&Rru(>D+GBtwG-o#^Rj6^8vP0YaL=$)D;YlC8q01tt*bGxX}LqKW0Pm zbfFH~N|!Q*(9lPY-72+HwB&Y;DZ>#uXoz7Z<1sUcb!jDvHg4`KBHn%c=uiHAo`2ys z50F;y$nF3AFMmxz_M?|?-mIf*fuA3GfQO@B;=$hbc>2IsE=twu-H-Bc`NzLu1{NRt z;YUnJ|E1SHR+m!Dn%sc6vD`I12hc;ePwHgrXo~O-YH4_#dl|Y7Jm0P7a65rGeO>+e z#KNGUT$$c;BXczpsIxS6fhID4;f+0bV(<1u<^Gt@$2iBKmJHJ6K+!|d+q*G23kubC z91OA(FyoR$6@#Ww4lC+}j+o{wxF$wDXamuq2|591Lm=ZuQYgRb`VLNV=7;pEk}{AJ zHh5A{8*DuQ2oS`Szqe|G=1R3&ZeBri&MZsapsDr&wI2;sg;52hiLz4E(BIaEojIuJ zjsZW?S`}h5BwpY0N<@YjxGMs`JJrz$-teon)~jyV^7g$u6k_TfqtQ=i(E!H`gMafUV@bzG6Xc@S=m zJvtN#h-3!VC|krmC1L&6(FO&0WJmb* zsEfPOrBHXmY@rvUDDuo~g<}afF$v4qUQD2pF40%a1C69^V8tt+8Lb>f5Nhq1soFBf`RHAQVheWq*ES9*jz0s zID2z+7q>v<`b<$rd0ur?VB@6p^^w8H^s`GcU17T~Pt&_Mq zS~7ixi;U0qjg+S}hnKIDH(Wtsk3p9V(+)j6DkxXVH9c)~{Wd3aK&R)(DpnC}!6z zNQT*2!}V}@W9YxZ4O=lZrC|f%wlJI^zEiCjYK@(GcyVH3c+SaFd<2+{>wu`JGB3*A zqk;xxYZM<`;E+o*MW}Vd0EV~V+qrs&E|)D5+9i9gy0<%ko)lTR^_#=h0~SmGgm#M) zF1t}njo3Ok@W!`%WWpvi!(jUe#ucMf9_4NLjn!oEp?s}C?FW63B+#OxjVkv_p}zQQ zaE*llz_6T71clKO&on_PgAySAs|muXu`hr%uXH6y27t|)QS{b9IGkZXl48lFc#yv{ z#(Y$5z=s~lo&D_1!>{wC@r)n;x<(OaH-*}&hd=)62Y=4KDHO6gn{WQ(|LV`qzK=If zJ}#WH_r-txxBf2A3HQxw8cyW)$=b1+C1F01j)n<6X<*()Vo7f7ZuUA|BI4K8W;3r&LsYW)BkToPs^K@!);Q2a(inc#?e8-F;F|_F#qLjwm0xw50 zxSk;b7AcjFE7`$r3l?d53bv$amLCtC?cFL6spC_CKq*)Ti{xMpcQhHEfGt$X2*dRR z*ff4qG*Uq7`PsN&j0nb3{a7~$IE#AR@S4en+7@L82dx4x14-fV0QK5~6;~mMkc!;P z`;7m%2IxYyzk5^-4vrBFtuS7(fYlMM+B}zPJe0LWz#gO@lzdC}V^Sh4L)un#)=IQ` z)~`5rTg=R7&z}?jyf(dIMQoM zfXzC#<0tTj_0L#jw2iLZYDPQoVsenMVqUTEl|7(M9EKB^doqaASX_TH9!=0H_Ilcw zl7-nwOGtuYgay7z+{%JOkflCyj!;bw0=1m5f8nPkWLFM?ZwMAKtFLq^wAWHmOIZcZ zCJ7nuM57(LJFv(xNYT_zpoCQA!`>c|gzN`ps6X?Aw`T9APb;(F7Jcjk-!DD;TGf*q zDyQfLx36-)z4-_K$bQA+zxwUJ$)kLMz$s5~zxE{_>@DX>iOsB{!b*vu)+JsgcQNCz z?3mefoy@!8fdZ8TXk~XF3XRiNvXZ?~K@*o(`}1Jd;kpTG^58Q0F~f#E27AS5Wisv< zcA{aLq3L)qXF=4Oq;CYR0RI6*)R3i4BnrM&!YsPj(dNw+yPAGxO|{^hLR1+}+U+{Y z$1GHHTyq4>3N-bOoS$d%br(?^z#jTY+D09$jqM9AI(U+@8Ax1Wm7j_NP)&)#6<8pc z$VUi*fdK$QfI=^_uk))!zGpr)XMv9I0LIswwd9VIw}d#qa5xGF8@66!^E@@1+nZ0F z9JW+JZHfmv>bOam+Y3c@OpR`F9Nr6cH{)^UxHO}{=ABzXLS%`B!Xs+RDabrkNdUi% zqcVi0XY8wj}41xz6VPU+RtDNWIV5GMv$P+KG+ z;$IL~u5=~Dyk0U_B=I26M%_9_Tb2>jTKR!gGuZenhT@Lw%S-R)VRk!%!hOQN_t^cP zs-Al3$V;A-2H$1w1|aA2;*|Tp{4I^oaTDh5d)yxovf|*L3diRYef!;%hs!_sa>Nt2 zcesDI4q<=qZ}G(!z9;hSA8@y~hwJg=Z#*`8t6}dri&A3@8)7syFTM#eElZnl8O=F2 z#C0G;ozV2Wf~!v1Wl)d;)fAu$i;P&9FcZV>o2&l9Ck00wtbT*l-w-Bs(7)t*Xr&X7 zBH)DX4;(6Nt|G2rDx7qxzuECIY>M;8bC#THZr?eZ5%W63qO0IG#SX{swrb>X1&Fjr zqD8QzC8-O)_=cvM-aeJE7OGglw=Oha!<}?ciU~vh)Dt2ZdFdIFjfoo?*`I>64K7h# zi5rtPrB%m#)ofbE^C1q!UtqkFq}5$3#1a?Tv9+01FGvS+^Ac1U@;GjAYD29p$KOu+>1u@L$Z0(w-YMg>776LW@4sAK2}E-5yLIj200(A=(I84W@D0G6d0 z2-y_;8!!)wu9cbr!D`*cix(m4R=N~8uVdiRkwMfo^>7#;;6ECr+ap3MMxM?{AFLhy z*ufwD#>}FANL7OO9N4el$CGTim>zx*y=1Md{q1guOo0hcOBmAVp>l-wL`P({4ci+) zmT9^*Vh9Q-kEo}qTu@UKKqIHnOD0*^D90p=JHY%C`q!Fb!2((%e)VMWj1&n0x~b66 z%nvqGBu4`m0*|430mySi#V8_0m4tVRKp76S=rUc4GX#@6o8sm#D8sWj zF_?RbH`*FFu$FSYd{;lL2jJ6sDc*&X$%N4jD=%G}>5H(99S(#~z;RYk&eFFGSe+b^ zD>(KkvlyQB46z&Ztr)BX0XwP)ghuEf&Mn&!Q@MLRQ&fAIXoXs)XHlIOVx#qN&qnv! zkc2m`qV{8^1J@sjhcVTs<(_P~&Rci3W3++co~^Of?gV;DZTOey@cG4-f|ZTW}%}KseuxP6GnAS}0<_3dSYHgr|=YRj_?()uBVrW6JWKGb_c z8Zi^N)*(w7kpV(!G$~?6#{94Z&_AET&3BiVN%{k=AM4M~rOwWV zPI7}4=O;BqmW7lbD_shuc*4f{BnB=cSZYP`L0(@oVN1+zqk7?4^=zw$)Wglr27&y~ z@HMwx`LgEsx$($76V_;_;Fb<8uh7916t~$uGkG$;f%chdm-sR0gQmClt*YqMWXpzM zox-;7aXQ_r8Vk~RvU6z1qduUY(Ei9J(jJ%}JlNC?)#5_4eZD?Ic?72H1zbWf zW8oyt*be3fU076!u|Be6BiJS;X_r+s2c+A{{8igGXXfYx3DzSMso6zzEDx38?P$RA zn6($^nqPnQJfl3^y?w%D%5&4quDCVPPcehR!~w;D9%29(jF)B`bq%0DbcX3NnT#OV zA7PXQ0vG<9u?k;I*_FyoVJCDJPl@>mopfVf&ImGRpn5dw@S8YV6_jmn%}SsoHA$FP~?DGn|2Iv9c5 z>OWfYbbfo+ICJ$UHljQm?&)u?kU=ieU`^I-t~>+i{6tNIXVkQ`pPk)Y=Z=M6n$9Ym z?FB@p05%=%;zfzs!rnL{d2sCz%TeeK;V}g4Wmb^{7Xema^PzT<8v#rXryaq0XfiEU z+Y#|GZ3Oo~M#IwvCRo4 zJ)Fl7K*FU*0EOgw5@#UgI=MeI9ZetH-EB(z-GGvU^R_=Mo<+%Sy@>MM;oX~P_%T^=#TY4!t|>K74To7e8Pio+$j`CTrBI*_Q-33zPi292 z`DmPtpshLUs-zotf66*``}nR(i}v<`n%wJyeECoocH+I~q*IU6IBC2aI_uWz7b;!BK;-2=SX8(nm|$ z>7>oCD3)Y6*k+YYv~fMd%bNLfoz%7sO8~1ZnJ=T&PQV0=XFlR!z#Aw+kPOn`quvpZ zLrr!v5(M>SK%7Xp&R`)>LeQE1Br>={_Ny_`_p)uajwUBWP|41YR;;T|>|k1!+57|w?uXS7uT z146tG91z4Qtc2?l&T8JSg{v!gzy?}`O|XA1N}YxISuZfFhzE~QCI-UVqH8sP50aa{ zsYZ~$OA)Mig5l5u&0K&uAXE#9@`TV0B7sCC;xhzGRS!CVvV*=mI;I3-e3cWJ-36~y z!z+)dLbWG`!M<&$14IzkKCV|RXTS`a97P&k#VC^t+6~w591@faZPK}LYYiju3i3&S ziE#aRaJWOX#F|wgU?zF)K>%4upm=_3odsp?)`Godzl|nmbbgTt83md3O2Jp}Ws1A- zpKWzcON0M(iB@Xa-(X0XzE~cV8PH2RX&#zCGa8g&!b0S+Jk|^yHwZIE6n9hX8%1<9 z+p{$HOXzmz%E30tj5f+q5kQ7H*{XaRd=O(njDakw8W+q>M#xVA*`V)-^mvU-h!!Ce zR=Siip#<3I*sq<9nd;Dhb;yi0p^lp2HJ}VAl>wC1o~gNNG$9F+a5eHnCh4;y^+>Or z96{|+x@m3$WpDOw<9imdQKA$nHyeEF?^L2|_-RwX9s zJE=({gy1$-IDNK|&j6lmK8DY6Mi8^Cw3`&A5jvhP;z-Xwm7z zPK+q?b=%lTM8uS!M#B(GoC_pEWKD#5eDKi5q#(Fp7lP3bsqq{-3jJs|ikVI(Vn6Nx z1(KDpxmVz5CGD^Ut&uV*@-hb8&A1=4 z13>AZdmUq{a0E~>QA>?X5zt*87Y^u}r#gFZ7RXCz%MS1gn5boK+OO&mZHzwI2ZO;8 zrv-E2jpc6##7GVxUhxi}^}?t{gp>V6CWhvzB!$ry&_C?BWc<()x70ci!@!o-RR|js>`UcoASTG3^|JiWzTjn9O|ttm2t+T19TVEsoT#sp`ILOPXQQ+NX?i!yk11c z0>k&OR~nBpKTA1qxnd0({j662QXN%sFV7&s=`sxfJFo+k?#3dOw9P^TKIH?vJ#~*E z(e`$-Xu;N<?vb4vkK6?gZ`{h!$?e;9ftT*! zJU?2)qn{*G`N6!1OKY{so#N;GP2<-}4aD?wosAob5oD+|#%GvR~ z?E`(bv5p8GOv9mjg zh~$5HdSGbB`u$1_Tpny9-FKieP{)+4#?>h6cR>hhvMgUU-xhZ&_v+Q(jC{N z_0B$1d(0>h^jJxgo(kX0wp_J)c)7ZJpzcnKrVWj~oKFw|@2*>c)NZ^CJgg`lxAK}y zT;{0&?v;|sLlB*47eS1|Lx?HPtC&g0eiK(95KDfVV-1xhl}t=y;eck-%)nEVqmDXC z-X?1xRMthi{u;e0W8{t-G>Cv%t;iYCy}_O=fg!WL>z!V+<6yw+Qjp_}5D@L(iLpu# zOB|0)nE8bJ`~tH_Q4R&^7Ti#(DwCF`6-DN>WYPu@dzMZ*e&Gyio#WpO)`0j-rUa)86m9!mDUYBgErVeS23ndW-Y_8gDs2G|#}Q*~S+v1ZZ;Db)hD6PBp|yaGK&`VxkN zm^wVj3I2_2L~SI8EsxS(L|vm(LaiYobEmbP(|Q!ThR8T7?bAW#5O$Tq=AFyBBkV+0(a9F+kKIP`CtgaYXwnJqgfi)(-Rm(-2I7u2QR3{pn9~{$V4L_! zSXvu8ta@6nP4~&)s+LNmyAO{deDMKS;tG%aGd$bV@BzWjk^t*dTVb1OY?^(Rk0@Pol+E#vw5p#1pE4FJ_6z1 zms&KlOp4>6K*ji5P)Rsa%OE4E!J-w^pvYR&s{eGBQ4YhGNN>Vat+kw)x-mu1NnB-> zM}@qMJ2_X+gvD&8ZeBYkW|(vconqqQHOI-X;?Q?zqlf16v~h{`v-=F9*l?wtM8wQA zEeqXh|1hf=B=h8H2p$G@T(++A`{tcn6cuhI8x>rbUInuP?JHI?r^TBWAWS&O0dQ6D zeB)}q%|_r2&4UN0y;s0fRBfRlqd6CP4jB$zXrTfr>XcfcnYsYNGKR_~fIh0-W-~3o zZSZ?a%HU5dskMhQxIv(mnoTw}V7x%JZm}eTIL$-pvh{1TrtEZ{pTVKf_l_x zIArHahP_tCfL6}*zmld_jq z2VCYXG4{LZJKJ||;T^^s(w9l8DQp#t@rjW_4hgZNf>wB3g_;JObRCk~km*KNCQwpn zOchWP&P+rWOsq0$&;AVaB%Yx52=0Sa-S{J9&sy#8wm(|wN+L{b8g9gnKT5B8$xbxh z3C0Y%jfjC~u#fSmrp5b&AdC;q+SI&A9=rJRtB(pe`OkmdzJK%PJzji@`>XX7xVqjN z(|#@a)OIz}3$g8g;_D|5KKQs`j`e|5Pc1q=;NZlTmffw?NNG8-*cfPO zI=q?ac-T~I)djkg71Nx=I4kS)+`36Ktl5t?h-px*&^#Mn9UmX)6T%`dhJs3{?j!Q- zFw7#KLY-$cLf6{bo$Br_gG@oX$>6cs7w+r4Y5#4%|>jyO{_Ku1SFge+hvzdRU2w1Vy?>FE6c~r3M7Jed;+L{^YNI zhsPhHowx4=8=^P;#GSiV>DpKQ(mW$Q#=ktoMdu_pG2P-fx4uJ#R+SJdP^gV zny3;U$1Ryd;e^zUV^Z+=>h#c5@!KD^8ZPEY59WKsw+L7#gd1FnL#78m3@I%Ah(QS% z~qKd%NUiYe|11v1O$L{B!kgz$CBWBC;=)2 z1_}DW`K2ijr=9N8EI)8o2hyhPWO!g#1cYnV0IsuTcC))jj5F7r*V=1mr~Ru#JhHTs z&B3yLCKp4^z)zdz2I5BEq23DRjJFYtizE>;8=h|<2mm`cZyLketgJwK3E$0>x$xaV3XZeovkmrqR&ZVEQUoZ4 z^*228*?x3w#-ZS`T;a13kgYGJZ^NmQTIp&_qQ-AeO*L zl673r-qg-}(I zXlM-B6Siy4tShNxE@gVyN3kpukJ5H9!ADheT{~6TEN4v9d}>a3b7*x(TyugJ<*;ZM zeSvk$Wnif~5O$MFN|%{t&we$K3aF^KY?X*+h5^SVVvQQW7T}n`-K#)TOxQ}82G{^R z&3SZsUyRuw^yqh55M8_Wx|mCn^6+{&}*Yc(hW9$6*SmzS_$q3 zZZW#6&;pVe(Ss)G7Q^2-FDB&#-qN(QlPYV8+|zb6^PBt@tuto;B#V;DHTTm+Q?lQD z0qjNH2X|`Jq#!iN7{oOjy&W8npoLi$-|(SzGfOW73v7&X(-t)EAQ=);g@s|A>Mm30 zWGPfFLQVcN!-da*Xu>B&EQ>zY&6M`?#)PsrdZyu43HDtcPHZ()JgS^bE~qQe;Lr7C9m)X_#3F@K?GPI(4;Ln`B0?F0^t7xzzwpc~t*k>5oW-#`(9Q8KEPtmeX(@ew+$*5=z7 z?qaonc-jnNwbv5Nak9LFE`vPalNcKvmE)XciWRPnsCH7Il2VVKf{5r9oX`kq*Mzv7jZOgKR*s?G~5eRe(VQAx;#mM0BJO zX9imt3${}0|D|&fmcaDoTwi71!D){I~7%dmhjXEVt}WeuKyNy|nYc z{fgRuk;tUq;?C`V{be2=`|Kb5_9s4aYxG4m3+=6b^}_6po1&Sndx&ZtzA7@8`=3>V z@aE0hJDTOYL0Peel9WWe1(7%1Yn5I1!Pgxi5+%8kmY>M>?n1E6y9Qd~n3$e#yauJIIhyS=2&ILU!x&81qLemY4Ci*+O~#m7(29f8w0T~n^tU91Alf6L zj`IKlaX5TM1|Ko_-8q{Fbs5>oLe$Ti;}}i61!T}4kVPgx2QB313 z&|TPREpFzX8pVzo1IL9158vKj8w2FNUYiuXKk5x|zIH@6o|_)-_RmdtF^+y|WpzQ1 z)E^7n2<-$~e3%yTjneF55-&jre#G2p8JH->4ITDp5EW;gj_q+c-UY=+2todeySM=IJ2P z=Df{jpzVs8&>vo9nx}niiA++bwlp(bfF*Y$y?dv>0E;XtfXf&_6rmd@Q5hATeU+*U z`0SW&FT0pxx$@4?e?`wtAGHSF^elEEcgbLDO9oKI#(}K~TR;JTM#q(D))H3&s$Mkp zy0*r) z3F^2Rgr1T(A%H{XhHMN7A-LDA6vhjhsnFt^{;5wMTn7%(B5{e8E(Ml0WMnrWJh%N* z8#vHBO-XdOeUF*ICd;Mq5^Ovx^w2Lov8RaPI`;$aKl`|VKR+i2e{*AUyViFQv&My+ z`{G^W_OgAlkHQ$K>H5(1LmvO|R~v0k)pld`W*`5|Hjk3lx%~Dqk5}se!7LMeue%eo zwX%G7CVI1Hy`{;7XOa<&&t>;j+5O1j9}`LxBUxsHI4VRN4ozkRU{@L*yFvoC1%rrH)=dKq^o114GqWMw<*E5J-RNLOI45&-r?Lb9sEr+};OkiP zFaYK?Q2sX7{NDgrTe4)klZl%TUaj6{xf*y{d&we@3(IDbSh8V5juasY3`I7^1r6=3 z3Jer>qcqAWzLC;&NhInxn$>Yv?KbLo)`Z`y=Tohix3gd~;5LD179Bfa;KI%)$<9U?=+cAWS$9=jbVMwujgQ8ZF z(Zkz$Z+w2TX3mII=2ZUxO@ z=~IPG2tj3cL7{x5D?tc_%FW4^ZZCfQcmJV% z^^rgQM|$t2fBE0fL?rqZfJjxk8YFv#KwddzM$v#lI4Z0F_(1$^LE4N2IpbFhxypXD+-yz0;Cc*G zhT;2;(5it*$7XfA`;d$t+t7?Zt060Z^!#DGslTrN(5twr+nZP4JfWBGx>{POG*GkV z18MtO3ZziL;}I64!G`;%)i}GS}TCA5X(jMPICZJour9Rxm(Y-qnBDa z6BRgS4_p~|0Igduv+(>waU_!8uqUf>>l6* zrlj(O*&|cm)B6Tu7ic-lC)v*sx2;$6;$vMcNAxVhESMKPBTypa4VsT zIRWd#{5qH8p352Qv@X-h5cV?;q@9kJY0zJj&vVv{atpGjyx+gopf*_#{K45{4(L1w zQV>~Z*`9}zqTD^c)6a$(rOQ<^iGY=$_m?sdc_=B?6GI^-&Cz6p5mg>`UFP#gW;U`5 zvzvAY@@iE%vrzW>DJ=qkb(SoGV0%0;S772_&z_o9x7qWBkxQaKUyp#gC)5_TuNhU} zfie(rLv0Thm4SMYW`k`Jg$cn%#*khq=tDM^wJXrlfSpdU1a%E`WePtl=OYta%n496 zD?RxAOM3Wt7`p2meOSf7r7+w(;WN?5HEj_>NL4q8y@3t5D7ZG#Y^6(~&KHKsM&OK6 z#n(nQG)cBWDIn!p_wezV(Z=n6@!K{mc<{XsyygXuewD9OqOiB;cX-(Nh9o zBN5@wS)#d^Z${&=zlQ15_5NIxrq8BK{9A^NN0X47+ zE0>fM&@gUkk6RdIh-@G>O%!q$>u0pfodcW3fVYNi9E~VYnEa}%9@uAgk4Q=nH!ttg zsCN%*0}}0rS~HJQTZTpFbV0e6gU;Az0U#+a3HNofvXB|t3S8$TfLA2%w#OZ27RN6% zqg1wjLmt+yZv#so%2TZ$U2nb416+VNuErBMKEaUdS@n^j7-S{eW=3dRN!OsH7Jx3p zk{nEeNIhOiA_(13q259_18fF#LBdq}%)ZatHe1{9H%N4I=n06zO0|niu!ebsSR~q5 zU_Bu1QnZ6@3uj`H$wJ$<%B29>g&uq(^k%lh!?uMxez*fh9s*^!Wg4+aOk1P1`*ICU zy86DYcXxmEq4a#DPOuE!S@ZVtkAL%dYoxAhefM|&fqkhO?=#%5eC%sHie_1sZcl~X z>3ewK_UIFbkK0$uy10Gg$^$oVe&i#(dZ5m#CsQ1Z>r=Bf*hyLFeuaesuX z3;4Gclda|bE=^%BJVg@2%z3)|$&6uL1J~2<{{Rlu_=H8fX~8J02<-DbqgGRC?l6m) z7aoyRMlj>z>Lmm7~zZqN{kwg;0V#&>DYf|_7XWY!^h9o0$RI4~+P`)%)abc}Gg_+1-tNaMl2 zg%mTbDZAr%bMICG`49h$W_%Y{`h0q{kTFF@X4on9rBGBevl9HWXC=(Cg8S zIwaHqvz1}skWfQTFcKCL=)}nesrS_3DL%R8k!}pl%g0%d!YX=amiKuuuLp9UQlE)b z;c|=&JwXQafHefcP@J4FLpUT zl6X5EF1djeSG__p=TpR8<1G8VDcQ8B+P~bx=VHp6TQThmB5F^gR zUy?_7C}hWszRX({P_TB;-d&+)%7;io-)~!|LeBC<-E1LYgbYAm4>S?hM5$dTO*x&T zT6>-`^MR_Q9|D|RVps@_hb=IoDiHL*Mgi7}@*_$j5ILb%8|KnnCZd30XMU|jpw*qEA@G`;=u#$@W0>3_lXuQt z69Av6C)OCmPB&^PV>J6|xNiraB*0miW{UvA3mWZ2s7#4(Rn%)xD22wd3wGSE>#Xh) z?Lnz+@Q*;FUD1(2rszU0Ko zv2_3nLwsrweB!N=j0p%2TRzUs)`L$dPIZ>hr)znKdubl^PRD zrN|LLX$SBP0p*-dY{sO5Tsb+08IDL6CIu^938Y5goi-?~+oV7!!fUX5>XvQ?$SN?t zXE|ad7AaT$g^BgifBNh1ef^w}0TK7FKKTxhADr@FO80;M*M)IZsq1Shah-CX-M;zA z?B!?7H%mB{TRj0G#KM!W@$l6z?_J?Zd}(SKsL1v^|4}_#@xFKOKE>DE-p~EV{{au5 zfBCJa|NP(Mg&>sX-@Q7me=D=PH<)ZUm<0A^m;M@sKm%yC! z>GZIqCx48)-T49YkH_`MAv_F>G?3Md=ZbP5X`%3e@^vuWn3Nf+3kgAGML}nV5_6#i%=^qCQYSM@)~%&b!5@Tv*Y)AGiYc}+ z(?QAuSG|LZZ9NG-7>{@$Ntb0g*u-JCQTP-7D}h?3oI!ZksHjkH+KSjPDLq`d3c%9JtjU^qBgTNLhT=2$3WVl1Tz z>v~@6TF$|$ZjQmcQvv?B#AGfEQV>}jN6CbIn&3?cEcysJ^RRJLjw%L+V0!Ni(>F|P zb9>NCHMvVfAnpx--%M_}rt4+OwG^87X!BV}=FJJPQwEkPcc1~oSntv>j{Hz=x1Apj zIu&N_fV~COg(Npg*Mz_h;x;P7Av-UN0QvGDcARdliHMA$b5Ww^qYO7P$uzcr`f}zW zyqt-Z3B6^qbIu|^WtJcYi*Ro%T}s4%Ty|npUXjQ`W8#hIzUpsNYB; z0S0jF2-3OK3~7!f;~=9!!t8U$6d%lIB;ROYVd{0`o`c2rj{Q6X?7W0&>IsJBJ z3Kr#f^9t_EISnsjdpFHUZ+8(r5|S12zu|sJ=9TELVkAbf$|Arwp#^2!T@gw6jLA!fl&|e1Fm>nB!Xh=?XC}RS5;{7ujjfIz0^h&l+aY2`f z9zc@yL1iH;fHp}&72?gzQbJBb1MN`0M642)DL-Pj@-f^(+oO&|$BV{vSGp3qI8g%a zkzqc3U;ed6(te_ivlzfhxS_1!-d zNcKbAzfgyT=2vB=*5tXIoLTMQF4!w-rTYAU`$;ujL10l(J|OzrCvP@?26{sEOU>4{ zNb4_hD5hOovw#c%!_SqM-YwM!6MMnnx&=lfEAGvoKD@fF&)>d zSm?gRVbFXjQ&gQDG&BmHr;Z`rKLE{8=~fX^ci{w@I406)3ofG! z3=1OWr$cdpF~Q(mKX7YAuObK^LhOH-@$Y9bxW|$}20iSeMAx+S{lL`doG~|4B1B0a znoFT^x&<~DFT)5HXngnky$Ot%$XISM&&K*N-NwK>n=BvYVC^ZbOjNKqTTe*hvWoYa z^iEixn7OVoq!Qq!$5~1|h3=ri0X6GWA&5x;%Jq8?ONJq2UJ&<@jsQ4ALN0=bv$Z~T zci5lr4i65yjzflBp|@k^TmvN~fi-AuU{49^IfzW+lS8PPdRa0wV+wAG*{95>mYcv) z!Vbav7KH6#33n<|8qg5S`yyT5z(GZJI4CT?(xpI=0a4GF4c9v^)md6%a@tts{2`lat&e|78118?&Mx36;l_}=)2 z{_L%2&%Vjy*G^RS*%e~yOZLN+kU&K9hZNbI;hA%;LtShNn(8*WrhJTgGd^0XLp!kE zn`m#w5oIcaZhfXMpqfd#y{f@qG&zB)G?(D>)=4 z@I{V^<*ZPg1}__CofHaDWMxKk&ZtYm%*V7n9gH-b%}<*yDN?= zxUvTdf=r^;8OXwA8m{7l&IGU^X91!C3kMy(D$rD{FzP6RXPMW*;z^9Cvq}cb;6!?3 zTJh1q7_bT6Yblpi5oJ#?=XGj|_ii=n`gCv7By{Z!1hfAtY`U7ZnR)9#798mOa>4E| zen|Pk!TUeYyeMm{z!OF89T{`)P$_cP1M|e><42W>y~%? zNnYEV#`?MaLi)#l@9d3Np1--FWQE(O>{DIv{{Cu;y!@RyC(^iIofkc-syw&vKFq^M z1=s&h{HTiaUwH6qsC&A~eo*k$hg536@O2)zef`-#{IB2i-)q09oAmaJ_dodj z>%96B_t#o~z#B5>+%9o{kh?C#-ky2u0ag1|saH2)_vl6aQp;sap9<~Y=du0>cy&k1 z{eFD{``o#`W}kTU|DV79EB7CJ^V3>LVR--O-}w%Y4|D&>Pd~3^&GdS9yL`jG48MBs zQC-k-HA}v~{w0rjDGV^TZ$0$ll)t`$+#cU7h_R+5-u}3er133z%*ANOtUuYKd2*-5N_?dpLxw7os?um9TL;MFVl zYbr1=oQrXWsldhw;RhuKa4jil@DROv9;jhKN`w_A5um~qDwOM%C9P-w3l4Htx)e|I zw!~(2JWgPyZr2EEW!S*B0bhi1BWH&qg0F7Zx9w8>$*<{TJg#%`&;C;$ex~B#-v9C+ z^7ymdxt-qPA^)7djoaJFBl-Yd`ND;-YLnBB`mMju^Z)r*{+nOYX@B`MuM0-U?KSQ{ zx|rH8{_lVHzvabEb-Q%`9(SFcPjY|b1s=Y3fA_(ghZleL8%KQ64wG^_*K%MwHccTE ze54(Wxi`hU%JbRSK%tW0SJ}P_r^Sh>=$Wx9eMuXvrO9y4AoFGw?_g46)yxUay?x%Jl=t)BM@H?5Db`x(X|e!)ikO_)YF4?1XQ;a z@4$5kHQnwi457}Le*kqh3Q^o;1_SaZayWK>N{$GYnwh{9ZXG{`zJO7H@-F+q5QQI1>5X6iU_k51*9uyWRv?dat|-OuX%=@>P00=HcVd z@$lu#nauDj|JQH){S%(K$#7ogeqLQ?dwc!f16z;3@=?B!Yqz)avsb)$?Bl?Vk?m< zGBZtYN{?nV?q7ohva-R8cMSdF>Tp73L!_kxA#vDZk5y?pEO00zZ~zn%B8^Jglh9fQ$4}E zj{IvrO}}Jk1iJ)m+#b1i=3}qeKeLZtwol22F7QGD;yvy!g*xi?_CEWhpP;|k0;7EJY5RwDsLdCj zY-_l+;h(u{E z%vd~~Gi;h_SJ4G%_FHp^UwC7WcIDlk15AO&g3<@;5^NyIPSa3C2~J^x#sav1RBwk) zFO@z76R12wD@-w}V5FI`KfG;Kp(U2Yk>3@6KC&vfZq0Xhzt}?tsh+CmmvDV7rW;xg zF08)~2PkRH8F8rXHbHkc&0=dB&9MCe_E8@|JDp4;7^asX8)^^RKTuzwZVyqllz;<# zsW8{$5-Czjv>CV)W=^9H&Bb0H`y>-MFs9@tfRO-_x!76<(uqnudA#8j!#l3mxqR+) zzqli~Y6&Ze)~9wSW-C4gds*36GmjZ8vp;9p^4VI_JN?TRfJKTD_$Lb{l5h>oyJR_q zuL2D!YNixwqYe;ra>(Y<6*}z}u6Fmn?ZV7XnzLFoGuiu#_SbqNx}q`Ndt5Nq@^|9XQKH+P8oYmY93ZNiLWJiAH+p8ceeam)j%jHfA7c1+0-R@Hn+^ zP4@IEEl@50Nm*WAOvS99l;w|4G)6|L`zK|22NV61vb+i+KPk(LkXX~3f2n18*J?*F z_gdsnqmN_Wx#g$yIlT?9%nCSrLBb* zbubETHYY=YCl$F)knDH>wLNO4vs5SvHa0)mPAiU3w~=E8!Dw-~d1gxhlT4bG`lB2% z9MkgLlXj(qR4oRjfo7u&eqz7T=&{!0a!!)*3K2N~AY;s6)ec6$)l=XP1{&ptHVx|0 z4FQjiYpFp;&EyPM?M64yWDbSSrmD1NvT>Cu)#=gZFde~+UwRy!}O?lr=#zIQKD};W-&Eb&56s!~3s+7R3*zy>ncJR*$)5s`s7Ci+g zRhHPi?w)E0;hfz=ft0awzOO%31sb(p=loiBx(CU@C3rKMs(qb$T^9F$wcpPGmr@E{ zmavj}4X+U;r9!3_JE}DrOB=^8kPQ==Ug8%;>2(s;fs6t{fL1oow9fr?>Z3xCISB%sWIP{LlBbz TYy!lDGL{dSu>4K*;Sc|RJ0qld literal 0 HcmV?d00001 diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_timestamp.out b/regression-test/data/external_table_p2/hudi/test_hudi_timestamp.out new file mode 100644 index 000000000000000..dc47ff86d90a8d7 --- /dev/null +++ b/regression-test/data/external_table_p2/hudi/test_hudi_timestamp.out @@ -0,0 +1,6 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !timestamp -- +20241115015956800 20241115015956800_0_2 1 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 1 Alice 2024-10-25T08:00 +20241115015956800 20241115015956800_0_0 2 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 2 Bob 2024-10-25T09:30:00 +20241115015956800 20241115015956800_0_1 3 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 3 Charlie 2024-10-25T11:00:00 + diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_timetravel.out b/regression-test/data/external_table_p2/hudi/test_hudi_timetravel.out index 38b6ff7846f49ae..a9b5d23595a8e3d 100644 --- a/regression-test/data/external_table_p2/hudi/test_hudi_timetravel.out +++ b/regression-test/data/external_table_p2/hudi/test_hudi_timetravel.out @@ -1,125 +1,121 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !q00 -- -20240724195843565 20240724195843565_0_0 20240724195843565_0_0 PAR1=para/par2=para 7a788b37-9ef0-409a-bc42-6793e35fcad5-0_0-129-127_20240724195848377.parquet 1 a b para para -20240724195845718 20240724195845718_0_0 20240724195845718_0_0 PAR1=para/par2=parb fef19b36-4a18-4d8c-b204-1ed448f7de51-0_0-152-155_20240724195850799.parquet 2 a b para parb -20240724195848377 20240724195848377_0_1 20240724195848377_0_0 PAR1=para/par2=para 7a788b37-9ef0-409a-bc42-6793e35fcad5-0_0-129-127_20240724195848377.parquet 3 a b para para -20240724195850799 20240724195850799_0_1 20240724195850799_0_0 PAR1=para/par2=parb fef19b36-4a18-4d8c-b204-1ed448f7de51-0_0-152-155_20240724195850799.parquet 4 a b para parb +-- !timetravel1 -- +1000 --- !q01 -- +-- !timetravel2 -- +2000 --- !q02 -- +-- !timetravel3 -- +3000 --- !q01 -- -20240724195843565 20240724195843565_0_0 20240724195843565_0_0 PAR1=para/par2=para 7a788b37-9ef0-409a-bc42-6793e35fcad5-0_0-129-127_20240724195848377.parquet 1 a b para para -20240724195845718 20240724195845718_0_0 20240724195845718_0_0 PAR1=para/par2=parb fef19b36-4a18-4d8c-b204-1ed448f7de51-0_0-152-155_20240724195850799.parquet 2 a b para parb -20240724195848377 20240724195848377_0_1 20240724195848377_0_0 PAR1=para/par2=para 7a788b37-9ef0-409a-bc42-6793e35fcad5-0_0-129-127_20240724195848377.parquet 3 a b para para -20240724195850799 20240724195850799_0_1 20240724195850799_0_0 PAR1=para/par2=parb fef19b36-4a18-4d8c-b204-1ed448f7de51-0_0-152-155_20240724195850799.parquet 4 a b para parb - --- !q02 -- -20240724195843565 20240724195843565_0_0 20240724195843565_0_0 PAR1=para/par2=para 7a788b37-9ef0-409a-bc42-6793e35fcad5-0_0-129-127_20240724195848377.parquet 1 a b para para -20240724195845718 20240724195845718_0_0 20240724195845718_0_0 PAR1=para/par2=parb fef19b36-4a18-4d8c-b204-1ed448f7de51-0_0-152-155_20240724195850799.parquet 2 a b para parb -20240724195848377 20240724195848377_0_1 20240724195848377_0_0 PAR1=para/par2=para 7a788b37-9ef0-409a-bc42-6793e35fcad5-0_0-129-127_20240724195848377.parquet 3 a b para para -20240724195850799 20240724195850799_0_1 20240724195850799_0_0 PAR1=para/par2=parb fef19b36-4a18-4d8c-b204-1ed448f7de51-0_0-152-155_20240724195850799.parquet 4 a b para parb - --- !q03 -- - --- !q04 -- - --- !q05 -- -1 a b para para - --- !q06 -- -1 a b para para - --- !q07 -- -1 a b para para -2 a b para parb - --- !q08 -- -1 a b para para -2 a b para parb - --- !q09 -- -1 a b para para -2 a b para parb -3 a b para para - --- !q10 -- -1 a b para para -2 a b para parb -3 a b para para - --- !q11 -- -1 a b para para -2 a b para parb -3 a b para para -4 a b para parb - --- !q12 -- -1 a b para para -2 a b para parb -3 a b para para -4 a b para parb - --- !q50 -- -20240724195853736 20240724195853736_0_0 Id:1 PAR1=para/par2=para c5a8ebb7-f929-43ba-9f8d-d733fae27605-0_0-203-210_20240724195858450.parquet 1 a b para para -20240724195856338 20240724195856338_0_0 Id:2 PAR1=para/par2=parb 23756678-cf81-481c-b559-85c0b47b0a80-0_0-219-228_20240724195902682.parquet 2 a b para parb -20240724195858450 20240724195858450_0_1 Id:3 PAR1=para/par2=para c5a8ebb7-f929-43ba-9f8d-d733fae27605-0_0-203-210_20240724195858450.parquet 3 a b para para -20240724195902682 20240724195902682_0_1 Id:4 PAR1=para/par2=parb 23756678-cf81-481c-b559-85c0b47b0a80-0_0-219-228_20240724195902682.parquet 4 a b para parb - --- !q51 -- - --- !q52 -- - --- !q51 -- -20240724195853736 20240724195853736_0_0 Id:1 PAR1=para/par2=para c5a8ebb7-f929-43ba-9f8d-d733fae27605-0_0-203-210_20240724195858450.parquet 1 a b para para -20240724195856338 20240724195856338_0_0 Id:2 PAR1=para/par2=parb 23756678-cf81-481c-b559-85c0b47b0a80-0_0-219-228_20240724195902682.parquet 2 a b para parb -20240724195858450 20240724195858450_0_1 Id:3 PAR1=para/par2=para c5a8ebb7-f929-43ba-9f8d-d733fae27605-0_0-203-210_20240724195858450.parquet 3 a b para para -20240724195902682 20240724195902682_0_1 Id:4 PAR1=para/par2=parb 23756678-cf81-481c-b559-85c0b47b0a80-0_0-219-228_20240724195902682.parquet 4 a b para parb - --- !q52 -- -20240724195853736 20240724195853736_0_0 Id:1 PAR1=para/par2=para c5a8ebb7-f929-43ba-9f8d-d733fae27605-0_0-203-210_20240724195858450.parquet 1 a b para para -20240724195856338 20240724195856338_0_0 Id:2 PAR1=para/par2=parb 23756678-cf81-481c-b559-85c0b47b0a80-0_0-219-228_20240724195902682.parquet 2 a b para parb -20240724195858450 20240724195858450_0_1 Id:3 PAR1=para/par2=para c5a8ebb7-f929-43ba-9f8d-d733fae27605-0_0-203-210_20240724195858450.parquet 3 a b para para -20240724195902682 20240724195902682_0_1 Id:4 PAR1=para/par2=parb 23756678-cf81-481c-b559-85c0b47b0a80-0_0-219-228_20240724195902682.parquet 4 a b para parb - --- !q53 -- - --- !q54 -- - --- !q55 -- -1 a b para para - --- !q56 -- -1 a b para para - --- !q57 -- -1 a b para para -2 a b para parb - --- !q58 -- -1 a b para para -2 a b para parb - --- !q59 -- -1 a b para para -2 a b para parb -3 a b para para - --- !q60 -- -1 a b para para -2 a b para parb -3 a b para para - --- !q61 -- -1 a b para para -2 a b para parb -3 a b para para -4 a b para parb - --- !q62 -- -1 a b para para -2 a b para parb -3 a b para para -4 a b para parb +-- !timetravel4 -- +4000 + +-- !timetravel5 -- +5000 + +-- !timetravel6 -- +6000 + +-- !timetravel7 -- +7000 + +-- !timetravel8 -- +8000 + +-- !timetravel9 -- +9000 + +-- !timetravel10 -- +10000 + +-- !timetravel1 -- +1000 + +-- !timetravel2 -- +2000 + +-- !timetravel3 -- +3000 + +-- !timetravel4 -- +4000 + +-- !timetravel5 -- +5000 + +-- !timetravel6 -- +6000 + +-- !timetravel7 -- +7000 + +-- !timetravel8 -- +8000 + +-- !timetravel9 -- +9000 + +-- !timetravel10 -- +10000 + +-- !timetravel1 -- +1000 + +-- !timetravel2 -- +2000 + +-- !timetravel3 -- +3000 + +-- !timetravel4 -- +4000 + +-- !timetravel5 -- +5000 + +-- !timetravel6 -- +6000 + +-- !timetravel7 -- +7000 + +-- !timetravel8 -- +8000 + +-- !timetravel9 -- +9000 + +-- !timetravel10 -- +10000 + +-- !timetravel1 -- +1000 + +-- !timetravel2 -- +2000 + +-- !timetravel3 -- +3000 + +-- !timetravel4 -- +4000 + +-- !timetravel5 -- +5000 + +-- !timetravel6 -- +6000 + +-- !timetravel7 -- +7000 + +-- !timetravel8 -- +8000 + +-- !timetravel9 -- +9000 + +-- !timetravel10 -- +10000 diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_catalog.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_catalog.groovy new file mode 100644 index 000000000000000..f2082ef89c7a50f --- /dev/null +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_catalog.groovy @@ -0,0 +1,39 @@ +// 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_catalog", "p2,external,hudi,external_remote,external_remote_hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHudiTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable hudi test") + } + + String catalog_name = "test_hudi_catalog" + String props = context.config.otherConfigs.get("hudiEmrCatalog") + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog if not exists ${catalog_name} properties ( + ${props} + ); + """ + + sql """ switch ${catalog_name};""" + sql """ use regression_hudi;""" + sql """ set enable_fallback_to_original_planner=false """ + def tables = sql """ show tables; """ + assertTrue(tables.size() > 0) + sql """drop catalog if exists ${catalog_name};""" +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_incremental.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_incremental.groovy new file mode 100644 index 000000000000000..8cc1d2a852b8c4b --- /dev/null +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_incremental.groovy @@ -0,0 +1,111 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_hudi_incremental", "p2,external,hudi,external_remote,external_remote_hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHudiTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable hudi test") + } + + String catalog_name = "test_hudi_incremental" + String props = context.config.otherConfigs.get("hudiEmrCatalog") + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog if not exists ${catalog_name} properties ( + ${props} + ); + """ + + sql """ switch ${catalog_name};""" + sql """ use regression_hudi;""" + sql """ set enable_fallback_to_original_planner=false """ + + def test_hudi_incremental_querys = { table_name, timestamps -> + timestamps.eachWithIndex { timestamp, index -> + def query_name = "qt_incremental_${index + 1}_end" + "${query_name}" """ select count(user_id) from ${table_name}@incr('beginTime' = '${timestamp}'); """ + query_name = "qt_incremental_earliest_${index + 1}" + "${query_name}" """ select count(user_id) from ${table_name}@incr('beginTime' = 'earliest', 'endTime' = '${timestamp}'); """ + if (index > 0) { + query_name = "qt_incremental_${index}_${index + 1}" + "${query_name}" """ select count(user_id) from ${table_name}@incr('beginTime' = '${timestamps[index - 1]}', 'endTime' = '${timestamp}'); """ + } + } + } + + // spark-sql "select distinct _hoodie_commit_time from user_activity_log_cow_non_partition order by _hoodie_commit_time;" + def timestamps_cow_non_partition = [ + "20241114151946599", + "20241114151952471", + "20241114151956317", + "20241114151958164", + "20241114152000425", + "20241114152004116", + "20241114152005954", + "20241114152007945", + "20241114152009764", + "20241114152011901", + ] + test_hudi_incremental_querys("user_activity_log_cow_non_partition", timestamps_cow_non_partition) + + // spark-sql "select distinct _hoodie_commit_time from user_activity_log_cow_partition order by _hoodie_commit_time;" + def timestamps_cow_partition = [ + "20241114152034850", + "20241114152042944", + "20241114152052682", + "20241114152101650", + "20241114152110650", + "20241114152120030", + "20241114152128871", + "20241114152137714", + "20241114152147114", + "20241114152156417", + ] + test_hudi_incremental_querys("user_activity_log_cow_partition", timestamps_cow_partition) + + // spark-sql "select distinct _hoodie_commit_time from user_activity_log_mor_non_partition order by _hoodie_commit_time;" + def timestamps_mor_non_partition = [ + "20241114152014186", + "20241114152015753", + "20241114152017539", + "20241114152019371", + "20241114152020915", + "20241114152022911", + "20241114152024706", + "20241114152026873", + "20241114152028770", + "20241114152030746", + ] + test_hudi_incremental_querys("user_activity_log_mor_non_partition", timestamps_mor_non_partition) + + // spark-sql "select distinct _hoodie_commit_time from user_activity_log_mor_partition order by _hoodie_commit_time;" + def timestamps_mor_partition = [ + "20241114152207700", + "20241114152214609", + "20241114152223933", + "20241114152232579", + "20241114152241610", + "20241114152252244", + "20241114152302763", + "20241114152313010", + "20241114152323587", + "20241114152334111", + ] + test_hudi_incremental_querys("user_activity_log_mor_partition", timestamps_mor_partition) + + sql """drop catalog if exists ${catalog_name};""" +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_schema_evolution.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_schema_evolution.groovy new file mode 100644 index 000000000000000..b247aaf492400d4 --- /dev/null +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_schema_evolution.groovy @@ -0,0 +1,48 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_hudi_schema_evolution", "p2,external,hudi,external_remote,external_remote_hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHudiTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable hudi test") + } + + String catalog_name = "test_hudi_schema_evolution" + String props = context.config.otherConfigs.get("hudiEmrCatalog") + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog if not exists ${catalog_name} properties ( + ${props} + ); + """ + + sql """ switch ${catalog_name};""" + sql """ use regression_hudi;""" + sql """ set enable_fallback_to_original_planner=false """ + + qt_adding_simple_columns_table """ select * from adding_simple_columns_table order by id """ + qt_altering_simple_columns_table """ select * from altering_simple_columns_table order by id """ + // qt_deleting_simple_columns_table """ select * from deleting_simple_columns_table order by id """ + // qt_renaming_simple_columns_table """ select * from renaming_simple_columns_table order by id """ + + qt_adding_complex_columns_table """ select * from adding_complex_columns_table order by id """ + qt_altering_complex_columns_table """ select * from altering_complex_columns_table order by id """ + // qt_deleting_complex_columns_table """ select * from deleting_complex_columns_table order by id """ + // qt_renaming_complex_columns_table """ select * from renaming_complex_columns_table order by id """ + + sql """drop catalog if exists ${catalog_name};""" +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_snapshot.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_snapshot.groovy new file mode 100644 index 000000000000000..53c09e6d5a90312 --- /dev/null +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_snapshot.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_snapshot", "p2,external,hudi,external_remote,external_remote_hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHudiTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable hudi test") + } + + String catalog_name = "test_hudi_snapshot" + String props = context.config.otherConfigs.get("hudiEmrCatalog") + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog if not exists ${catalog_name} properties ( + ${props} + ); + """ + + sql """ switch ${catalog_name};""" + sql """ use regression_hudi;""" + sql """ set enable_fallback_to_original_planner=false """ + + // 创建groovy函数,接收table_name为参数 + def test_hudi_snapshot_querys = { table_name -> + // Query users by event_time in descending order and limit output + qt_q01 """SELECT * FROM ${table_name} ORDER BY event_time DESC LIMIT 10;""" + + // Query all active user records and limit output + qt_q02 """SELECT * FROM ${table_name} WHERE is_active = TRUE ORDER BY event_time LIMIT 10;""" + + // Query specific user's activity records and limit output + qt_q03 """SELECT * FROM ${table_name} WHERE user_id = '62785e0e-ad44-4321-8b20-9ee4c4daca4a' ORDER BY event_time LIMIT 5;""" + + // Query events within a specific time range and limit output + qt_q04 """SELECT * FROM ${table_name} WHERE event_time BETWEEN '2024-01-01 00:00:00' AND '2024-12-31 23:59:59' ORDER BY event_time LIMIT 10;""" + + // Count users by age group and limit output + qt_q05 """SELECT age, COUNT(*) AS user_count FROM ${table_name} GROUP BY age ORDER BY user_count DESC LIMIT 5;""" + + // Query users with purchase records and limit output + qt_q06 """SELECT user_id, purchases FROM ${table_name} WHERE array_size(purchases) > 0 ORDER BY user_id LIMIT 5;""" + + // Query users with a specific tag and limit output + qt_q07 """SELECT * FROM ${table_name} WHERE array_contains(tags, 'others') ORDER BY event_time LIMIT 5;""" + + // Query users living in a specific city and limit output + qt_q08 """SELECT * FROM ${table_name} WHERE struct_element(address, 'city') = 'North Rachelview' ORDER BY event_time LIMIT 5;""" + + // Query users within a specific coordinate range and limit output + qt_q09 """SELECT * FROM ${table_name} WHERE struct_element(struct_element(address, 'coordinates'), 'latitude') BETWEEN 0 AND 100 AND struct_element(struct_element(address, 'coordinates'), 'longitude') BETWEEN 0 AND 100 ORDER BY event_time LIMIT 5;""" + + // Query records with ratings above a specific value and limit output + qt_q10 """SELECT * FROM ${table_name} WHERE rating > 4.5 ORDER BY rating DESC LIMIT 5;""" + + // Query all users' signup dates and limit output + qt_q11 """SELECT user_id, signup_date FROM ${table_name} ORDER BY signup_date DESC LIMIT 10;""" + + // Query users with a specific postal code and limit output + qt_q12 """SELECT * FROM ${table_name} WHERE struct_element(address, 'postal_code') = '80312' ORDER BY event_time LIMIT 5;""" + + // Query users with profile pictures and limit output + qt_q13 """SELECT user_id, profile_picture FROM ${table_name} WHERE profile_picture IS NOT NULL ORDER BY user_id LIMIT 5;""" + + // Query users by signup date and limit output + qt_q14 """SELECT * FROM ${table_name} WHERE signup_date = '2024-01-15' ORDER BY user_id LIMIT 5;""" + + // Query the total count of purchases for each user and limit output + qt_q15 """SELECT user_id, array_size(purchases) AS purchase_count FROM ${table_name} ORDER BY purchase_count DESC LIMIT 5;""" + } + + test_hudi_snapshot_querys("user_activity_log_cow_non_partition") + test_hudi_snapshot_querys("user_activity_log_cow_partition") + test_hudi_snapshot_querys("user_activity_log_mor_non_partition") + test_hudi_snapshot_querys("user_activity_log_mor_partition") + + sql """drop catalog if exists ${catalog_name};""" +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_timestamp.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_timestamp.groovy new file mode 100644 index 000000000000000..c1ba630e4a7d01e --- /dev/null +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_timestamp.groovy @@ -0,0 +1,62 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_hudi_timestamp", "p2,external,hudi,external_remote,external_remote_hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHudiTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable hudi test") + } + + String catalog_name = "test_hudi_timestamp" + String props = context.config.otherConfigs.get("hudiEmrCatalog") + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog if not exists ${catalog_name} properties ( + ${props} + ); + """ + + sql """ switch ${catalog_name};""" + sql """ use regression_hudi;""" + sql """ set enable_fallback_to_original_planner=false """ + + // TODO: fix hudi timezone issue and enable this + // qt_timestamp """ select * from hudi_table_with_timestamp order by id; """ + + sql """drop catalog if exists ${catalog_name};""" +} + +// DROP TABLE IF EXISTS hudi_table_with_timestamp; + +// -- create table +// CREATE TABLE hudi_table_with_timestamp ( +// id STRING, +// name STRING, +// event_time TIMESTAMP +// ) USING HUDI +// OPTIONS ( +// type = 'cow', +// primaryKey = 'id', +// preCombineField = 'event_time' +// ); + +// SET TIME ZONE 'America/Los_Angeles'; + +// INSERT OVERWRITE hudi_table_with_timestamp VALUES +// ('1', 'Alice', timestamp('2024-10-25 08:00:00')), +// ('2', 'Bob', timestamp('2024-10-25 09:30:00')), +// ('3', 'Charlie', timestamp('2024-10-25 11:00:00')); \ No newline at end of file diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_timetravel.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_timetravel.groovy index db535e3517987eb..4d458dc4381dcf7 100644 --- a/regression-test/suites/external_table_p2/hudi/test_hudi_timetravel.groovy +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_timetravel.groovy @@ -16,13 +16,6 @@ // under the License. suite("test_hudi_timetravel", "p2,external,hudi,external_remote,external_remote_hudi") { - - Boolean ignoreP2 = true; - if (ignoreP2) { - logger.info("disable p2 test"); - return; - } - String enabled = context.config.otherConfigs.get("enableExternalHudiTest") if (enabled == null || !enabled.equalsIgnoreCase("true")) { logger.info("disable hudi test") @@ -37,77 +30,76 @@ suite("test_hudi_timetravel", "p2,external,hudi,external_remote,external_remote_ ); """ - sql """switch ${catalog_name};""" + sql """ switch ${catalog_name};""" sql """ use regression_hudi;""" sql """ set enable_fallback_to_original_planner=false """ - qt_q00 """select * from timetravel_cow order by id""" - qt_q01 """select * from timetravel_cow FOR TIME AS OF "2024-07-24" order by id""" // no data - qt_q02 """select * from timetravel_cow FOR TIME AS OF "20240724" order by id""" // no data - qt_q01 """select * from timetravel_cow FOR TIME AS OF "2024-07-25" order by id""" - qt_q02 """select * from timetravel_cow FOR TIME AS OF "20240725" order by id""" - qt_q03 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "2024-07-24 19:58:43" order by id """ // no data - qt_q04 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "20240724195843" order by id """ // no data - qt_q05 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "2024-07-24 19:58:44" order by id """ // one - qt_q06 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "20240724195844" order by id """ //one - qt_q07 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "2024-07-24 19:58:48" order by id """ // two - qt_q08 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "20240724195848" order by id """ // two - qt_q09 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "2024-07-24 19:58:49" order by id """ // three - qt_q10 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "20240724195849" order by id """ // three - qt_q11 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "2024-07-24 19:58:51" order by id """ // four - qt_q12 """ select id, val1,val2,par1,par2 from timetravel_cow FOR TIME AS OF "20240724195851" order by id """ // four - - qt_q50 """select * from timetravel_mor order by id""" - qt_q51 """select * from timetravel_mor FOR TIME AS OF "2024-07-24" order by id""" // no data - qt_q52 """select * from timetravel_mor FOR TIME AS OF "20240724" order by id""" // no data - qt_q51 """select * from timetravel_mor FOR TIME AS OF "2024-07-25" order by id""" - qt_q52 """select * from timetravel_mor FOR TIME AS OF "20240725" order by id""" - qt_q53 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "2024-07-24 19:58:53" order by id """ // no data - qt_q54 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "20240724195853" order by id """ // no data - qt_q55 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "2024-07-24 19:58:54" order by id """ // one - qt_q56 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "20240724195854" order by id """ //one - qt_q57 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "2024-07-24 19:58:58" order by id """ // two - qt_q58 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "20240724195858" order by id """ // two - qt_q59 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "2024-07-24 19:58:59" order by id """ // three - qt_q60 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "20240724195859" order by id """ // three - qt_q61 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "2024-07-24 19:59:03" order by id """ // four - qt_q62 """ select id, val1,val2,par1,par2 from timetravel_mor FOR TIME AS OF "20240724195903" order by id """ // four -} - - -/* + def test_hudi_timetravel_querys = { table_name, timestamps -> + timestamps.eachWithIndex { timestamp, index -> + def query_name = "qt_timetravel${index + 1}" + "${query_name}" """ select count(user_id) from ${table_name} for time as of "${timestamp}"; """ + } + } -create table timetravel_cow ( - Id int, - VAL1 string, - val2 string, - PAR1 string, - par2 string -) using hudi -partitioned by (par1, par2) -TBLPROPERTIES ( - 'type' = 'cow'); + // spark-sql "select distinct _hoodie_commit_time from user_activity_log_cow_non_partition order by _hoodie_commit_time;" + def timestamps_cow_non_partition = [ + "20241114151946599", + "20241114151952471", + "20241114151956317", + "20241114151958164", + "20241114152000425", + "20241114152004116", + "20241114152005954", + "20241114152007945", + "20241114152009764", + "20241114152011901", + ] + test_hudi_timetravel_querys("user_activity_log_cow_non_partition", timestamps_cow_non_partition) -create table timetravel_mor ( - Id int, - VAL1 string, - val2 string, - PAR1 string, - par2 string -) using hudi -partitioned by (par1, par2) -TBLPROPERTIES ( - 'primaryKey' = 'Id', - 'type' = 'mor'); + // spark-sql "select distinct _hoodie_commit_time from user_activity_log_cow_partition order by _hoodie_commit_time;" + def timestamps_cow_partition = [ + "20241114152034850", + "20241114152042944", + "20241114152052682", + "20241114152101650", + "20241114152110650", + "20241114152120030", + "20241114152128871", + "20241114152137714", + "20241114152147114", + "20241114152156417", + ] + test_hudi_timetravel_querys("user_activity_log_cow_partition", timestamps_cow_partition) -insert into timetravel_cow values (1, 'a','b','para','para'); -insert into timetravel_cow values (2, 'a','b','para','parb'); -insert into timetravel_cow values (3, 'a','b','para','para'); -insert into timetravel_cow values (4, 'a','b','para','parb'); + // spark-sql "select distinct _hoodie_commit_time from user_activity_log_mor_non_partition order by _hoodie_commit_time;" + def timestamps_mor_non_partition = [ + "20241114152014186", + "20241114152015753", + "20241114152017539", + "20241114152019371", + "20241114152020915", + "20241114152022911", + "20241114152024706", + "20241114152026873", + "20241114152028770", + "20241114152030746", + ] + test_hudi_timetravel_querys("user_activity_log_mor_non_partition", timestamps_mor_non_partition) -insert into timetravel_mor values (1, 'a','b','para','para'); -insert into timetravel_mor values (2, 'a','b','para','parb'); -insert into timetravel_mor values (3, 'a','b','para','para'); -insert into timetravel_mor values (4, 'a','b','para','parb'); + // spark-sql "select distinct _hoodie_commit_time from user_activity_log_mor_partition order by _hoodie_commit_time;" + def timestamps_mor_partition = [ + "20241114152207700", + "20241114152214609", + "20241114152223933", + "20241114152232579", + "20241114152241610", + "20241114152252244", + "20241114152302763", + "20241114152313010", + "20241114152323587", + "20241114152334111", + ] + test_hudi_timetravel_querys("user_activity_log_mor_partition", timestamps_mor_partition) -*/ + sql """drop catalog if exists ${catalog_name};""" +} \ No newline at end of file From 9c1f6abe021d056f53268f5756dc0a4e95e0633b Mon Sep 17 00:00:00 2001 From: Socrates Date: Fri, 22 Nov 2024 14:38:08 +0800 Subject: [PATCH 040/110] [fix](meta-cache) fix refreshOnlyCatalogCache when use_meta_cache = false (#44363) ### What problem does this PR solve? Bug: When hive catalog set use_meta_cache=false, refresh catalog cannot update the database list. Fix: Set initialized = false in `refreshOnlyCatalogCache()`. --- .../doris/datasource/ExternalCatalog.java | 1 + .../hive/test_hive_use_meta_cache.out | 106 ++++++++++- .../hive/test_hive_use_meta_cache.groovy | 171 +++++++++++------- 3 files changed, 207 insertions(+), 71 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java index 091ee347b101551..cde08113373aee4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java @@ -420,6 +420,7 @@ private void refreshOnlyCatalogCache(boolean invalidCache) { if (useMetaCache.get() && metaCache != null) { metaCache.invalidateAll(); } else if (!useMetaCache.get()) { + this.initialized = false; for (ExternalDatabase db : idToDb.values()) { db.setUnInitialized(invalidCache); } diff --git a/regression-test/data/external_table_p0/hive/test_hive_use_meta_cache.out b/regression-test/data/external_table_p0/hive/test_hive_use_meta_cache.out index a5fc7ede82b8bec..4d54c601c876a24 100644 --- a/regression-test/data/external_table_p0/hive/test_hive_use_meta_cache.out +++ b/regression-test/data/external_table_p0/hive/test_hive_use_meta_cache.out @@ -28,15 +28,20 @@ test_use_meta_cache_db_hive test_use_meta_cache_tbl_hive -- !sql08 -- -test_use_meta_cache_tbl_hive -- !sql09 -- +p1=part1 +p1=part2 -- !sql10 -- -test_use_meta_cache_db_hive -- !sql11 -- +-- !sql12 -- +test_use_meta_cache_db_hive + +-- !sql13 -- + -- !sql01 -- -- !sql02 -- @@ -61,17 +66,114 @@ test_use_meta_cache_db_hive -- !sql05 -- -- !sql06 -- +test_use_meta_cache_tbl_hive -- !sql07 -- test_use_meta_cache_tbl_hive -- !sql08 -- + +-- !sql09 -- +p1=part1 +p1=part2 + +-- !sql10 -- +test_use_meta_cache_partitioned_tbl_hive +test_use_meta_cache_tbl_hive + +-- !sql11 -- + +-- !sql12 -- +test_use_meta_cache_db_hive + +-- !sql13 -- + +-- !sql01 -- + +-- !sql02 -- +test_use_meta_cache_db + +-- !sql03 -- +test_use_meta_cache_tbl + +-- !sql04 -- + +-- !sql05 -- + +-- !sql01 -- + +-- !sql02 -- + +-- !sql03 -- +test_use_meta_cache_db_hive + +-- !sql04 -- + +-- !sql05 -- + +-- !sql06 -- + +-- !sql07 -- test_use_meta_cache_tbl_hive +-- !sql08 -- + -- !sql09 -- +p1=part1 +p1=part2 -- !sql10 -- + +-- !sql11 -- + +-- !sql12 -- +test_use_meta_cache_db_hive + +-- !sql13 -- + +-- !sql01 -- + +-- !sql02 -- +test_use_meta_cache_db + +-- !sql03 -- +test_use_meta_cache_tbl + +-- !sql04 -- + +-- !sql05 -- + +-- !sql01 -- + +-- !sql02 -- + +-- !sql03 -- test_use_meta_cache_db_hive +-- !sql04 -- + +-- !sql05 -- + +-- !sql06 -- +test_use_meta_cache_tbl_hive + +-- !sql07 -- +test_use_meta_cache_tbl_hive + +-- !sql08 -- + +-- !sql09 -- +p1=part1 +p1=part2 + +-- !sql10 -- +test_use_meta_cache_partitioned_tbl_hive +test_use_meta_cache_tbl_hive + -- !sql11 -- +-- !sql12 -- +test_use_meta_cache_db_hive + +-- !sql13 -- + diff --git a/regression-test/suites/external_table_p0/hive/test_hive_use_meta_cache.groovy b/regression-test/suites/external_table_p0/hive/test_hive_use_meta_cache.groovy index 3562ce3126725de..df12fc74898f06f 100644 --- a/regression-test/suites/external_table_p0/hive/test_hive_use_meta_cache.groovy +++ b/regression-test/suites/external_table_p0/hive/test_hive_use_meta_cache.groovy @@ -23,80 +23,113 @@ suite("test_hive_use_meta_cache", "p0,external,hive,external_docker,external_doc return; } - for (String hivePrefix : ["hive3", "hive3"]) { + for (String hivePrefix : ["hive2", "hive3"]) { setHivePrefix(hivePrefix) try { - String hms_port = context.config.otherConfigs.get(hivePrefix + "HmsPort") - String hdfs_port = context.config.otherConfigs.get(hivePrefix + "HdfsPort") - String catalog = "test_${hivePrefix}_use_meta_cache" - String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + def test_use_meta_cache = { Boolean use_meta_cache -> + String hms_port = context.config.otherConfigs.get(hivePrefix + "HmsPort") + String hdfs_port = context.config.otherConfigs.get(hivePrefix + "HdfsPort") + String use_meta_cache_string = use_meta_cache ? "true" : "false" + String catalog = "test_${hivePrefix}_use_meta_cache_${use_meta_cache}" + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") - sql """drop catalog if exists ${catalog}""" - sql """create catalog if not exists ${catalog} properties ( - 'type'='hms', - 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', - 'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}', - 'use_meta_cache' = 'true' - );""" + sql """drop catalog if exists ${catalog}""" + sql """create catalog if not exists ${catalog} properties ( + 'type'='hms', + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', + 'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}', + 'use_meta_cache' = '${use_meta_cache_string}' + );""" + + // create from Doris, the cache will be filled immediately + String database= "test_use_meta_cache_db" + String table = "test_use_meta_cache_tbl" + String database_hive = "test_use_meta_cache_db_hive" + String table_hive = "test_use_meta_cache_tbl_hive" + String partitioned_table_hive = "test_use_meta_cache_partitioned_tbl_hive" + + sql "switch ${catalog}" + sql "drop database if exists ${database}" + sql "drop database if exists ${database_hive}" + order_qt_sql01 "show databases like '%${database}%'"; + sql "drop database if exists ${database}" + sql "create database ${database}" + order_qt_sql02 "show databases like '%${database}%'"; + sql "use ${database}" + sql "create table ${table} (k1 int)" + order_qt_sql03 "show tables" + sql "drop table ${table}" + order_qt_sql04 "show tables" + sql "drop database ${database}" + order_qt_sql05 "show databases like '%${database}%'"; - // create from Doris, the cache will be filled immediately - String database= "test_use_meta_cache_db" - String table = "test_use_meta_cache_tbl" - String database_hive = "test_use_meta_cache_db_hive" - String table_hive = "test_use_meta_cache_tbl_hive" - sql "switch ${catalog}" - sql "drop database if exists ${database}" - sql "drop database if exists ${database_hive}" - order_qt_sql01 "show databases like '%${database}%'"; - sql "drop database if exists ${database}" - sql "create database ${database}" - order_qt_sql02 "show databases like '%${database}%'"; - sql "use ${database}" - sql "create table ${table} (k1 int)" - order_qt_sql03 "show tables" - sql "drop table ${table}" - order_qt_sql04 "show tables" - sql "drop database ${database}" - order_qt_sql05 "show databases like '%${database}%'"; - - // create from Hive, the cache has different behavior - order_qt_sql01 "show databases like '%${database_hive}%'"; - hive_docker "drop database if exists ${database_hive}" - hive_docker "create database ${database_hive}" - // not see - order_qt_sql02 "show databases like '%${database_hive}%'"; - // but can use - sql "use ${database_hive}" - sql "refresh catalog ${catalog}" - // can see - order_qt_sql03 "show databases like '%${database_hive}%'"; - // show tables first to fill cache - order_qt_sql04 "show tables" - hive_docker "create table ${database_hive}.${table_hive} (k1 int)" - // not see - order_qt_sql05 "show tables" - // but can select - sql "select * from ${table_hive}" - // still not see - order_qt_sql06 "show tables" - sql "refresh database ${database_hive}" - // can see - order_qt_sql07 "show tables" - hive_docker "drop table ${database_hive}.${table_hive}" - // still can see - order_qt_sql08 "show tables" - sql "refresh database ${database_hive}" - // can not see - order_qt_sql09 "show tables" - hive_docker "drop database ${database_hive}" - // still can see - order_qt_sql10 "show databases like '%${database_hive}%'"; - sql "refresh catalog ${catalog}" - // can not see - order_qt_sql11 "show databases like '%${database_hive}%'"; + // create from Hive, the cache has different behavior + order_qt_sql01 "show databases like '%${database_hive}%'"; + hive_docker "drop database if exists ${database_hive}" + hive_docker "create database ${database_hive}" + // not see + order_qt_sql02 "show databases like '%${database_hive}%'"; + if (use_meta_cache) { + // if use meta cache, can use + sql "use ${database_hive}" + sql "refresh catalog ${catalog}" + } else { + // if not use meta cache, can not use + sql "refresh catalog ${catalog}" + sql "use ${database_hive}" + } + + // can see + order_qt_sql03 "show databases like '%${database_hive}%'"; + // show tables first to fill cache + order_qt_sql04 "show tables" + hive_docker "create table ${database_hive}.${table_hive} (k1 int)" + // not see + order_qt_sql05 "show tables" + if (use_meta_cache) { + // but can select + sql "select * from ${table_hive}" + // still not see + order_qt_sql06 "show tables" + sql "refresh database ${database_hive}" + } else { + // if not use meta cache, can not select + sql "refresh database ${database_hive}" + sql "select * from ${table_hive}" + order_qt_sql06 "show tables" + } + // can see + order_qt_sql07 "show tables" + + // test Hive Metastore table partition file listing + hive_docker "create table ${database_hive}.${partitioned_table_hive} (k1 int) partitioned by (p1 string)" + sql "refresh catalog ${catalog}" + order_qt_sql08 "show partitions from ${partitioned_table_hive}" + hive_docker "alter table ${database_hive}.${partitioned_table_hive} add partition (p1='part1')" + hive_docker "alter table ${database_hive}.${partitioned_table_hive} add partition (p1='part2')" + // can see because partition file listing is not cached + order_qt_sql09 "show partitions from ${partitioned_table_hive}" + + // drop tables + hive_docker "drop table ${database_hive}.${partitioned_table_hive}" + hive_docker "drop table ${database_hive}.${table_hive}" + // still can see + order_qt_sql10 "show tables" + sql "refresh database ${database_hive}" + // can not see + order_qt_sql11 "show tables" + + // drop database + hive_docker "drop database ${database_hive}" + // still can see + order_qt_sql12 "show databases like '%${database_hive}%'"; + sql "refresh catalog ${catalog}" + // can not see + order_qt_sql13 "show databases like '%${database_hive}%'"; + } + test_use_meta_cache(true) + test_use_meta_cache(false) } finally { } } } - - From 5edc7f959a9437c6ca9cda676e17223de39e6f69 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Fri, 22 Nov 2024 14:53:14 +0800 Subject: [PATCH 041/110] [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 | 7 ++++--- .../main/java/org/apache/doris/catalog/OlapTable.java | 8 +++++--- .../doris/datasource/hive/HMSExternalTable.java | 10 ++++++---- .../doris/datasource/paimon/PaimonExternalTable.java | 9 ++++++--- .../java/org/apache/doris/mtmv/MTMVPartitionUtil.java | 9 +++++---- .../mtmv/MTMVRelatedPartitionDescInitGenerator.java | 3 ++- .../org/apache/doris/mtmv/MTMVRelatedTableIf.java | 11 ++++++++--- .../org/apache/doris/mtmv/MTMVPartitionUtilTest.java | 5 +++-- 8 files changed, 39 insertions(+), 23 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 955bfd4279fd5c5..825bdef9f09819f 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 @@ -59,6 +59,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; @@ -363,7 +364,7 @@ public MTMVRefreshSnapshot getRefreshSnapshot() { * @return mvPartitionName ==> mvPartitionKeyDesc */ public Map generateMvPartitionDescs() throws AnalysisException { - Map mtmvItems = getAndCopyPartitionItems(); + Map mtmvItems = getAndCopyPartitionItems(OptionalLong.empty()); Map result = Maps.newHashMap(); for (Entry entry : mtmvItems.entrySet()) { result.put(entry.getKey(), entry.getValue().toPartitionKeyDesc()); @@ -392,7 +393,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()); @@ -425,7 +426,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 8a9ce4a2a2a5452..547a520a061e63c 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 @@ -110,6 +110,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; @@ -3260,7 +3261,7 @@ public PartitionType getPartitionType() { } @Override - public Map getAndCopyPartitionItems() throws AnalysisException { + public Map getAndCopyPartitionItems(OptionalLong snapshotId) throws AnalysisException { if (!tryReadLock(1, TimeUnit.MINUTES)) { throw new AnalysisException("get table read lock timeout, database=" + getDBName() + ",table=" + getName()); } @@ -3284,7 +3285,8 @@ public List getPartitionColumns() { } @Override - public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context) + public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, + OptionalLong snapshotId) throws AnalysisException { Map partitionVersions = context.getBaseVersions().getPartitionVersions(); long partitionId = getPartitionOrAnalysisException(partitionName).getId(); @@ -3294,7 +3296,7 @@ public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshCont } @Override - public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context) { + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, OptionalLong snapshotId) { Map tableVersions = context.getBaseVersions().getTableVersions(); long visibleVersion = tableVersions.containsKey(id) ? tableVersions.get(id) : getVisibleVersion(); return new MTMVVersionSnapshot(visibleVersion, id); 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 aacd9268ae35cfc..98984467d75b5c6 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 @@ -84,6 +84,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; @@ -748,7 +749,7 @@ public Set getPartitionColumnNames() { } @Override - public Map getAndCopyPartitionItems() { + public Map getAndCopyPartitionItems(OptionalLong snapshotId) { HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() .getMetaStoreCache((HMSExternalCatalog) getCatalog()); HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( @@ -763,8 +764,8 @@ public Map getAndCopyPartitionItems() { } @Override - public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context) - throws AnalysisException { + public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, + OptionalLong snapshotId) throws AnalysisException { HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() .getMetaStoreCache((HMSExternalCatalog) getCatalog()); HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( @@ -776,7 +777,8 @@ public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshCont } @Override - public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context) throws AnalysisException { + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, OptionalLong snapshotId) + throws AnalysisException { if (getPartitionType() == PartitionType.UNPARTITIONED) { return new MTMVMaxTimestampSnapshot(getName(), getLastDdlTime()); } 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 5645c4e89e726c5..632a0da0ebd316e 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 @@ -67,6 +67,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; import java.util.stream.Collectors; @@ -312,7 +313,7 @@ public void beforeMTMVRefresh(MTMV mtmv) throws DdlException { } @Override - public Map getAndCopyPartitionItems() { + public Map getAndCopyPartitionItems(OptionalLong snapshotId) { return Maps.newHashMap(getPartitionInfoFromCache().getNameToPartitionItem()); } @@ -333,7 +334,8 @@ public List getPartitionColumns() { } @Override - public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context) + public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, + OptionalLong snapshotId) throws AnalysisException { PaimonPartition paimonPartition = getPartitionInfoFromCache().getNameToPartition().get(partitionName); if (paimonPartition == null) { @@ -343,7 +345,8 @@ public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshCont } @Override - public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context) throws AnalysisException { + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, OptionalLong snapshotId) + throws AnalysisException { return new MTMVVersionSnapshot(getLatestSnapshotIdFromCache()); } 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 1cfb5e021a5309c..1bbc51fb004c573 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 @@ -50,6 +50,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Objects; +import java.util.OptionalLong; import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -329,7 +330,7 @@ public static boolean isSyncWithPartitions(MTMVRefreshContext context, String mt } for (String relatedPartitionName : relatedPartitionNames) { MTMVSnapshotIf relatedPartitionCurrentSnapshot = relatedTable - .getPartitionSnapshot(relatedPartitionName, context); + .getPartitionSnapshot(relatedPartitionName, context, OptionalLong.empty()); if (!mtmv.getRefreshSnapshot() .equalsWithRelatedPartition(mtmvPartitionName, relatedPartitionName, relatedPartitionCurrentSnapshot)) { @@ -446,7 +447,7 @@ private static boolean isSyncWithBaseTable(MTMVRefreshContext context, String mt if (!baseTable.needAutoRefresh()) { return true; } - MTMVSnapshotIf baseTableCurrentSnapshot = baseTable.getTableSnapshot(context); + MTMVSnapshotIf baseTableCurrentSnapshot = baseTable.getTableSnapshot(context, OptionalLong.empty()); return mtmv.getRefreshSnapshot() .equalsWithBaseTable(mtmvPartitionName, new BaseTableInfo(baseTable), baseTableCurrentSnapshot); } @@ -482,7 +483,7 @@ private static MTMVRefreshPartitionSnapshot generatePartitionSnapshot(MTMVRefres MTMVRelatedTableIf relatedTable = mtmv.getMvPartitionInfo().getRelatedTable(); for (String relatedPartitionName : relatedPartitionNames) { MTMVSnapshotIf partitionSnapshot = relatedTable - .getPartitionSnapshot(relatedPartitionName, context); + .getPartitionSnapshot(relatedPartitionName, context, OptionalLong.empty()); refreshPartitionSnapshot.getPartitions() .put(relatedPartitionName, partitionSnapshot); } @@ -497,7 +498,7 @@ private static MTMVRefreshPartitionSnapshot generatePartitionSnapshot(MTMVRefres continue; } refreshPartitionSnapshot.addTableSnapshot(baseTableInfo, - ((MTMVRelatedTableIf) table).getTableSnapshot(context)); + ((MTMVRelatedTableIf) table).getTableSnapshot(context, OptionalLong.empty())); } return refreshPartitionSnapshot; } 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 13b58239376116e..ef3100dec4c7327 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 @@ -20,6 +20,7 @@ import org.apache.doris.common.AnalysisException; import java.util.Map; +import java.util.OptionalLong; /** * get all related partition descs @@ -29,6 +30,6 @@ public class MTMVRelatedPartitionDescInitGenerator implements MTMVRelatedPartiti @Override public void apply(MTMVPartitionInfo mvPartitionInfo, Map mvProperties, RelatedPartitionDescResult lastResult) throws AnalysisException { - lastResult.setItems(mvPartitionInfo.getRelatedTable().getAndCopyPartitionItems()); + lastResult.setItems(mvPartitionInfo.getRelatedTable().getAndCopyPartitionItems(OptionalLong.empty())); } } 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 4a8b14603ce4d67..e18784ae253a0fe 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 @@ -26,6 +26,7 @@ import java.util.List; import java.util.Map; +import java.util.OptionalLong; import java.util.Set; /** @@ -38,9 +39,10 @@ public interface MTMVRelatedTableIf extends TableIf { * Note: This method is called every time there is a refresh and transparent rewrite, * so if this method is slow, it will significantly reduce query performance * + * @param snapshotId * @return partitionName->PartitionItem */ - Map getAndCopyPartitionItems() throws AnalysisException; + Map getAndCopyPartitionItems(OptionalLong snapshotId) throws AnalysisException; /** * getPartitionType LIST/RANGE/UNPARTITIONED @@ -70,12 +72,14 @@ public interface MTMVRelatedTableIf extends TableIf { * If snapshots have already been obtained in bulk in the context, * the results should be obtained directly from the context * + * @param snapshotId * @param partitionName * @param context * @return partition snapshot at current time * @throws AnalysisException */ - MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context) throws AnalysisException; + MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, OptionalLong snapshotId) + throws AnalysisException; /** * getTableSnapshot @@ -83,11 +87,12 @@ public interface MTMVRelatedTableIf extends TableIf { * If snapshots have already been obtained in bulk in the context, * the results should be obtained directly from the context * + * @param snapshotId * @param context * @return table snapshot at current time * @throws AnalysisException */ - MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context) throws AnalysisException; + MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, OptionalLong snapshotId) 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 997385742dc09a1..96ac59b81216bca 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 @@ -35,6 +35,7 @@ import org.junit.Test; import java.util.List; +import java.util.OptionalLong; import java.util.Set; public class MTMVPartitionUtilTest { @@ -112,7 +113,7 @@ public void setUp() throws NoSuchMethodException, SecurityException, AnalysisExc minTimes = 0; result = true; - baseOlapTable.getTableSnapshot((MTMVRefreshContext) any); + baseOlapTable.getTableSnapshot((MTMVRefreshContext) any, (OptionalLong) any); minTimes = 0; result = baseSnapshotIf; @@ -132,7 +133,7 @@ public void setUp() throws NoSuchMethodException, SecurityException, AnalysisExc minTimes = 0; result = true; - baseOlapTable.getPartitionSnapshot(anyString, (MTMVRefreshContext) any); + baseOlapTable.getPartitionSnapshot(anyString, (MTMVRefreshContext) any, (OptionalLong) any); minTimes = 0; result = baseSnapshotIf; From 127e597b7c68cf843f6135fea544e235781d7592 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Fri, 22 Nov 2024 14:55:03 +0800 Subject: [PATCH 042/110] [case](mtmv)Cannot compare the number of replicas to 1, as the pipeline may force the number of replicas to be set (#44417) --- regression-test/suites/mtmv_p0/test_mtmv_property.groovy | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/regression-test/suites/mtmv_p0/test_mtmv_property.groovy b/regression-test/suites/mtmv_p0/test_mtmv_property.groovy index e0ffbca8dbf63c2..4104f1480e88b7f 100644 --- a/regression-test/suites/mtmv_p0/test_mtmv_property.groovy +++ b/regression-test/suites/mtmv_p0/test_mtmv_property.groovy @@ -60,7 +60,8 @@ suite("test_mtmv_property","mtmv") { def showCreateTableResult = sql """show create materialized view ${mvName}""" logger.info("showCreateTableResult: " + showCreateTableResult.toString()) - assertTrue(showCreateTableResult.toString().contains('tag.location.default: 1')) + // Cannot compare the number of replicas to 1, as the pipeline may force the number of replicas to be set + assertTrue(showCreateTableResult.toString().contains('tag.location.default:')) assertTrue(showCreateTableResult.toString().contains('"min_load_replica_num" = "-1"')) assertTrue(showCreateTableResult.toString().contains('"storage_medium" = "hdd"')) assertTrue(showCreateTableResult.toString().contains('"store_row_column" = "true"')) From d8215d7a2b0a74ba7df8f07230a22be4dcec042e Mon Sep 17 00:00:00 2001 From: zhangdong Date: Fri, 22 Nov 2024 15:29:06 +0800 Subject: [PATCH 043/110] [opt](table) Remove unnecessary locks (#44416) `OlapTable`.`getPartitionIds` not need get readLock, Because it only returns the ID list without any traversal or other operations. If the caller needs to keep the partition of the table unchanged after obtaining the ID list, they should lock it themselves --- .../src/main/java/org/apache/doris/catalog/OlapTable.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) 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 547a520a061e63c..ec3bd2acbc57d6b 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 @@ -1385,12 +1385,7 @@ public List getEqualPartitionNames(List partitionIds1, List } public List getPartitionIds() { - readLock(); - try { - return new ArrayList<>(idToPartition.keySet()); - } finally { - readUnlock(); - } + return new ArrayList<>(idToPartition.keySet()); } public Set getCopiedBfColumns() { From 1ffecfd196c0848faad031f498359c7f506688b5 Mon Sep 17 00:00:00 2001 From: Gabriel Date: Fri, 22 Nov 2024 15:48:08 +0800 Subject: [PATCH 044/110] [fix](local exchange) Fix unbalance data distribution (#44421) Follow-up for #44137 --- be/src/pipeline/local_exchange/local_exchanger.h | 16 +++++++--------- be/src/pipeline/pipeline_fragment_context.cpp | 2 +- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/be/src/pipeline/local_exchange/local_exchanger.h b/be/src/pipeline/local_exchange/local_exchanger.h index 4912ab3369815ba..f518e2649f89e65 100644 --- a/be/src/pipeline/local_exchange/local_exchanger.h +++ b/be/src/pipeline/local_exchange/local_exchanger.h @@ -53,7 +53,7 @@ class ExchangerBase { ExchangerBase(int running_sink_operators, int num_sources, int num_partitions, int free_block_limit) : _running_sink_operators(running_sink_operators), - _running_source_operators(num_partitions), + _running_source_operators(num_sources), _num_partitions(num_partitions), _num_senders(running_sink_operators), _num_sources(num_sources), @@ -202,10 +202,13 @@ struct BlockWrapper { class ShuffleExchanger : public Exchanger { public: ENABLE_FACTORY_CREATOR(ShuffleExchanger); - ShuffleExchanger(int running_sink_operators, int num_partitions, int free_block_limit) - : Exchanger(running_sink_operators, num_partitions, + ShuffleExchanger(int running_sink_operators, int num_sources, int num_partitions, + int free_block_limit) + : Exchanger(running_sink_operators, num_sources, num_partitions, free_block_limit) { - _data_queue.resize(num_partitions); + DCHECK_GT(num_partitions, 0); + DCHECK_GT(num_sources, 0); + _data_queue.resize(num_sources); } ~ShuffleExchanger() override = default; Status sink(RuntimeState* state, vectorized::Block* in_block, bool eos, @@ -217,10 +220,6 @@ class ShuffleExchanger : public Exchanger { ExchangeType get_type() const override { return ExchangeType::HASH_SHUFFLE; } protected: - ShuffleExchanger(int running_sink_operators, int num_sources, int num_partitions, - int free_block_limit) - : Exchanger(running_sink_operators, num_sources, num_partitions, - free_block_limit) {} Status _split_rows(RuntimeState* state, const uint32_t* __restrict channel_ids, vectorized::Block* block, LocalExchangeSinkLocalState& local_state); }; @@ -232,7 +231,6 @@ class BucketShuffleExchanger final : public ShuffleExchanger { : ShuffleExchanger(running_sink_operators, num_sources, num_partitions, free_block_limit) { DCHECK_GT(num_partitions, 0); - _data_queue.resize(std::max(num_partitions, num_sources)); } ~BucketShuffleExchanger() override = default; ExchangeType get_type() const override { return ExchangeType::BUCKET_HASH_SHUFFLE; } diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index 7572b20d34112ec..0775ef3fb19826b 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -747,7 +747,7 @@ Status PipelineFragmentContext::_add_local_exchange_impl( switch (data_distribution.distribution_type) { case ExchangeType::HASH_SHUFFLE: shared_state->exchanger = ShuffleExchanger::create_unique( - std::max(cur_pipe->num_tasks(), _num_instances), + std::max(cur_pipe->num_tasks(), _num_instances), _num_instances, use_global_hash_shuffle ? _total_instances : _num_instances, _runtime_state->query_options().__isset.local_exchange_free_blocks_limit ? cast_set( From 76c13320999da9070b45411361b011d265967f48 Mon Sep 17 00:00:00 2001 From: Gabriel Date: Fri, 22 Nov 2024 16:03:48 +0800 Subject: [PATCH 045/110] [minor](log) Add more information in error message (#44452) --- be/src/runtime/fragment_mgr.cpp | 20 +++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index 5a8ea2377aa80fc..16b75794511c06d 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -751,7 +751,9 @@ std::string FragmentMgr::dump_pipeline_tasks(TUniqueId& query_id) { if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { return q_ctx->print_all_pipeline_context(); } else { - return fmt::format("Query context (query id = {}) not found. \n", print_id(query_id)); + return fmt::format( + "Dump pipeline tasks failed: Query context (query id = {}) not found. \n", + print_id(query_id)); } } @@ -1266,8 +1268,10 @@ Status FragmentMgr::send_filter_size(const PSendFilterSizeRequest* request) { if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { query_ctx = q_ctx; } else { - return Status::EndOfFile("Query context (query-id: {}) not found, maybe finished", - queryid.to_string()); + return Status::EndOfFile( + "Send filter size failed: Query context (query-id: {}) not found, maybe " + "finished", + queryid.to_string()); } } @@ -1288,8 +1292,9 @@ Status FragmentMgr::sync_filter_size(const PSyncFilterSizeRequest* request) { if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { query_ctx = q_ctx; } else { - return Status::InvalidArgument("Query context (query-id: {}) not found", - queryid.to_string()); + return Status::InvalidArgument( + "Sync filter size failed: Query context (query-id: {}) not found", + queryid.to_string()); } } return query_ctx->runtime_filter_mgr()->sync_filter_size(request); @@ -1308,8 +1313,9 @@ Status FragmentMgr::merge_filter(const PMergeFilterRequest* request, if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { query_ctx = q_ctx; } else { - return Status::InvalidArgument("Query context (query-id: {}) not found", - queryid.to_string()); + return Status::InvalidArgument( + "Merge filter size failed: Query context (query-id: {}) not found", + queryid.to_string()); } } SCOPED_ATTACH_TASK(query_ctx.get()); From f9585093b53fb379175466bc99556d989bd093ae Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Fri, 22 Nov 2024 13:50:57 +0530 Subject: [PATCH 046/110] [Enhancement] (nereids)implement showDeleteCommand in nereids (#44296) Issue Number: close #42754 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 12 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/ShowDeleteCommand.java | 98 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../test_dml_delete_table_auth.groovy | 2 +- .../suites/show_p0/test_show_delete.groovy | 1 + 7 files changed, 119 insertions(+), 2 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDeleteCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index ea2958c6bc9f970..d0bd7a9738bcc40 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -206,6 +206,7 @@ supportedShowStatement | SHOW AUTHORS #showAuthors | SHOW DYNAMIC PARTITION TABLES ((FROM | IN) database=multipartIdentifier)? #showDynamicPartition | SHOW LAST INSERT #showLastInsert + | SHOW DELETE ((FROM | IN) database=multipartIdentifier)? #showDelete | SHOW ALL? GRANTS #showGrants | SHOW GRANTS FOR userIdentify #showGrantsForUser | SHOW VIEW @@ -295,7 +296,6 @@ unsupportedShowStatement sortClause? limitClause? #showLoad | SHOW EXPORT ((FROM | IN) database=multipartIdentifier)? wildWhere? sortClause? limitClause? #showExport - | SHOW DELETE ((FROM | IN) database=multipartIdentifier)? #showDelete | SHOW ALTER TABLE (ROLLUP | (MATERIALIZED VIEW) | COLUMN) ((FROM | IN) database=multipartIdentifier)? wildWhere? sortClause? limitClause? #showAlterTable diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index d3b63541d10df26..ac837018d3fd5f6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -208,6 +208,7 @@ import org.apache.doris.nereids.DorisParser.ShowCreateMaterializedViewContext; import org.apache.doris.nereids.DorisParser.ShowCreateProcedureContext; import org.apache.doris.nereids.DorisParser.ShowCreateTableContext; +import org.apache.doris.nereids.DorisParser.ShowDeleteContext; import org.apache.doris.nereids.DorisParser.ShowDynamicPartitionContext; import org.apache.doris.nereids.DorisParser.ShowFrontendsContext; import org.apache.doris.nereids.DorisParser.ShowGrantsContext; @@ -473,6 +474,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowCreateMaterializedViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateProcedureCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateTableCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowDeleteCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDynamicPartitionCommand; import org.apache.doris.nereids.trees.plans.commands.ShowFrontendsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowGrantsCommand; @@ -4350,6 +4352,16 @@ public LogicalPlan visitShowTableId(ShowTableIdContext ctx) { return new ShowTableIdCommand(tableId); } + @Override + public LogicalPlan visitShowDelete(ShowDeleteContext ctx) { + String dbName = null; + if (ctx.database != null) { + List nameParts = visitMultipartIdentifier(ctx.database); + dbName = nameParts.get(0); // only one entry possible + } + return new ShowDeleteCommand(dbName); + } + @Override public LogicalPlan visitShowPrivileges(ShowPrivilegesContext ctx) { return new ShowPrivilegesCommand(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 3a4484f81d01c4f..83b771cec24e9fc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -189,6 +189,7 @@ public enum PlanType { SHOW_CREATE_CATALOG_COMMAND, SHOW_CREATE_MATERIALIZED_VIEW_COMMAND, SHOW_CREATE_TABLE_COMMAND, + SHOW_DELETE_COMMAND, SHOW_DYNAMIC_PARTITION_COMMAND, SHOW_FRONTENDS_COMMAND, SHOW_GRANTS_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDeleteCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDeleteCommand.java new file mode 100644 index 000000000000000..32e8320ad602970 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDeleteCommand.java @@ -0,0 +1,98 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.RedirectStatus; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.proc.DeleteInfoProcDir; +import org.apache.doris.load.DeleteHandler; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; +import com.google.common.collect.Lists; + +import java.util.ArrayList; +import java.util.List; + +/** + * show delete command + */ +public class ShowDeleteCommand extends ShowCommand { + private String dbName; // if not given use current. + + /** + * constructor + */ + public ShowDeleteCommand(String dbName) { + super(PlanType.SHOW_DELETE_COMMAND); + this.dbName = dbName; + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + if (Strings.isNullOrEmpty(dbName)) { + dbName = ctx.getDatabase(); + if (Strings.isNullOrEmpty(dbName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); + } + } + Env env = Env.getCurrentEnv(); + Database db = env.getInternalCatalog().getDbOrAnalysisException(dbName); + long dbId = db.getId(); + + DeleteHandler deleteHandler = env.getDeleteHandler(); + List> deleteInfos = deleteHandler.getDeleteInfosByDb(dbId); + List> rows = Lists.newArrayList(); + for (List deleteInfo : deleteInfos) { + List oneInfo = new ArrayList(deleteInfo.size()); + for (Comparable element : deleteInfo) { + oneInfo.add(element.toString()); + } + rows.add(oneInfo); + } + return new ShowResultSet(getMetaData(), rows); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowDeleteCommand(this, context); + } + + public ShowResultSetMetaData getMetaData() { + ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); + for (String title : DeleteInfoProcDir.TITLE_NAMES) { + builder.addColumn(new Column(title, ScalarType.createVarchar(30))); + } + return builder.build(); + } + + @Override + public RedirectStatus toRedirectStatus() { + return RedirectStatus.FORWARD_NO_SYNC; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index fa5b9486ed5fc95..06bf620e7f24820 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -68,6 +68,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowCreateMaterializedViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateProcedureCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateTableCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowDeleteCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDynamicPartitionCommand; import org.apache.doris.nereids.trees.plans.commands.ShowFrontendsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowGrantsCommand; @@ -411,6 +412,10 @@ default R visitShowTrashCommand(ShowTrashCommand showTrashCommand, C context) { return visitCommand(showTrashCommand, context); } + default R visitShowDeleteCommand(ShowDeleteCommand showDeleteCommand, C context) { + return visitCommand(showDeleteCommand, context); + } + default R visitShowPrivilegesCommand(ShowPrivilegesCommand showPrivilegesCommand, C context) { return visitCommand(showPrivilegesCommand, context); } diff --git a/regression-test/suites/auth_call/test_dml_delete_table_auth.groovy b/regression-test/suites/auth_call/test_dml_delete_table_auth.groovy index 405e9c830ebe037..541902ebd68b92a 100644 --- a/regression-test/suites/auth_call/test_dml_delete_table_auth.groovy +++ b/regression-test/suites/auth_call/test_dml_delete_table_auth.groovy @@ -59,7 +59,7 @@ suite("test_dml_delete_table_auth","p0,auth_call") { sql """DELETE FROM ${dbName}.${tableName} WHERE id = 3;""" exception "denied" } - + checkNereidsExecute("show DELETE from ${dbName}") def del_res = sql """show DELETE from ${dbName}""" assertTrue(del_res.size() == 0) } diff --git a/regression-test/suites/show_p0/test_show_delete.groovy b/regression-test/suites/show_p0/test_show_delete.groovy index 9acae1a0bc376c9..5f06367c1364859 100644 --- a/regression-test/suites/show_p0/test_show_delete.groovy +++ b/regression-test/suites/show_p0/test_show_delete.groovy @@ -44,6 +44,7 @@ suite("test_show_delete") { // don't care nereids planner sql """ delete from ${tableName} PARTITION Mar where type ='2'""" sql """ delete from ${tableName} where type ='1'""" + checkNereidsExecute("show DELETE") def showDeleteResult = sql """ show delete""" //When we test locally, multiple history results will be included, so size will be >= 2 assert showDeleteResult.size() >= 0 From c1c9337f064cfc5b9311fff3cb02c2e9981fec23 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Fri, 22 Nov 2024 14:10:33 +0530 Subject: [PATCH 047/110] [Enhancement] (nereids)implement showBrokerCommand in nereids (#44424) Issue Number: close #42761 --- .../org/apache/doris/nereids/DorisParser.g4 | 3 +- .../nereids/parser/LogicalPlanBuilder.java | 8 ++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/ShowBrokerCommand.java | 82 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../manager/test_manager_interface_2.groovy | 1 + 6 files changed, 98 insertions(+), 2 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowBrokerCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index d0bd7a9738bcc40..0a784515dca0d14 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -201,9 +201,9 @@ supportedDropStatement ; supportedShowStatement - : SHOW (GLOBAL | SESSION | LOCAL)? VARIABLES wildWhere? #showVariables | SHOW AUTHORS #showAuthors + | SHOW BROKER #showBroker | SHOW DYNAMIC PARTITION TABLES ((FROM | IN) database=multipartIdentifier)? #showDynamicPartition | SHOW LAST INSERT #showLastInsert | SHOW DELETE ((FROM | IN) database=multipartIdentifier)? #showDelete @@ -310,7 +310,6 @@ unsupportedShowStatement | SHOW ALL PROPERTIES wildWhere? #showAllProperties | SHOW BACKUP ((FROM | IN) database=multipartIdentifier)? wildWhere? #showBackup | SHOW BRIEF? RESTORE ((FROM | IN) database=multipartIdentifier)? wildWhere? #showRestore - | SHOW BROKER #showBroker | SHOW RESOURCES wildWhere? sortClause? limitClause? #showResources | SHOW WORKLOAD GROUPS wildWhere? #showWorkloadGroups | SHOW TRASH (ON backend=STRING_LITERAL)? #showTrash diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index ac837018d3fd5f6..7c2f5f0c07020c9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -201,6 +201,7 @@ import org.apache.doris.nereids.DorisParser.SetVariableWithTypeContext; import org.apache.doris.nereids.DorisParser.ShowAuthorsContext; import org.apache.doris.nereids.DorisParser.ShowBackendsContext; +import org.apache.doris.nereids.DorisParser.ShowBrokerContext; import org.apache.doris.nereids.DorisParser.ShowConfigContext; import org.apache.doris.nereids.DorisParser.ShowConstraintContext; import org.apache.doris.nereids.DorisParser.ShowCreateCatalogContext; @@ -467,6 +468,7 @@ import org.apache.doris.nereids.trees.plans.commands.SetUserPropertiesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowAuthorsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowBackendsCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowBrokerCommand; import org.apache.doris.nereids.trees.plans.commands.ShowConfigCommand; import org.apache.doris.nereids.trees.plans.commands.ShowConstraintsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateCatalogCommand; @@ -4334,6 +4336,12 @@ public RecoverPartitionCommand visitRecoverPartition(RecoverPartitionContext ctx partitionName, partitionId, newPartitionName); } + @Override + public LogicalPlan visitShowBroker(ShowBrokerContext ctx) { + return new ShowBrokerCommand(); + } + + @Override public LogicalPlan visitDropRole(DropRoleContext ctx) { return new DropRoleCommand(ctx.name.getText(), ctx.EXISTS() != null); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 83b771cec24e9fc..f62e8ef77955af7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -185,6 +185,7 @@ public enum PlanType { DROP_SQL_BLOCK_RULE_COMMAND, SHOW_BACKENDS_COMMAND, SHOW_BLOCK_RULE_COMMAND, + SHOW_BROKER_COMMAND, SHOW_CONFIG_COMMAND, SHOW_CREATE_CATALOG_COMMAND, SHOW_CREATE_MATERIALIZED_VIEW_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowBrokerCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowBrokerCommand.java new file mode 100644 index 000000000000000..c4c624f140456ee --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowBrokerCommand.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.nereids.trees.plans.commands; + +import org.apache.doris.analysis.RedirectStatus; +import org.apache.doris.catalog.BrokerMgr; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import java.util.List; + +/** + * show broker command + */ +public class ShowBrokerCommand extends ShowCommand { + + /** + * constructor + */ + public ShowBrokerCommand() { + super(PlanType.SHOW_BROKER_COMMAND); + } + + public ShowResultSetMetaData getMetaData() { + ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); + for (String title : BrokerMgr.BROKER_PROC_NODE_TITLE_NAMES) { + builder.addColumn(new Column(title, ScalarType.createVarchar(30))); + } + return builder.build(); + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN) + && !Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), + PrivPredicate.OPERATOR)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN/OPERATOR"); + } + List> brokersInfo = Env.getCurrentEnv().getBrokerMgr().getBrokersInfo(); + // Only success + return new ShowResultSet(getMetaData(), brokersInfo); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowBrokerCommand(this, context); + } + + @Override + public RedirectStatus toRedirectStatus() { + if (ConnectContext.get().getSessionVariable().getForwardToMaster()) { + return RedirectStatus.FORWARD_NO_SYNC; + } else { + return RedirectStatus.NO_FORWARD; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 06bf620e7f24820..4cfa75a8efed802 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -61,6 +61,7 @@ import org.apache.doris.nereids.trees.plans.commands.SetUserPropertiesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowAuthorsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowBackendsCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowBrokerCommand; import org.apache.doris.nereids.trees.plans.commands.ShowConfigCommand; import org.apache.doris.nereids.trees.plans.commands.ShowConstraintsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateCatalogCommand; @@ -396,6 +397,10 @@ default R visitRecoverPartitionCommand(RecoverPartitionCommand recoverPartitionC return visitCommand(recoverPartitionCommand, context); } + default R visitShowBrokerCommand(ShowBrokerCommand showBrokerCommand, C context) { + return visitCommand(showBrokerCommand, context); + } + default R visitDropRoleCommand(DropRoleCommand dropRoleCommand, C context) { return visitCommand(dropRoleCommand, context); } diff --git a/regression-test/suites/manager/test_manager_interface_2.groovy b/regression-test/suites/manager/test_manager_interface_2.groovy index bd47324de1508cf..9c53941c1573927 100644 --- a/regression-test/suites/manager/test_manager_interface_2.groovy +++ b/regression-test/suites/manager/test_manager_interface_2.groovy @@ -292,6 +292,7 @@ suite('test_manager_interface_2',"p0") { } sql """ALTER SYSTEM ADD BROKER test_manager_broker "${address}:${notExistPort}";""" + checkNereidsExecute("show broker") result = sql """ show broker """ x = 0 logger.info("result = ${result}" ) From dd67d68ac515a6202b83ab0df6f5831dd52ecf97 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Fri, 22 Nov 2024 14:10:54 +0530 Subject: [PATCH 048/110] [Enhancement] (nereids)implement DropUserCommand in nereids (#44431) Issue Number: close #42615 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../apache/doris/mysql/privilege/Auth.java | 4 + .../nereids/parser/LogicalPlanBuilder.java | 8 ++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../trees/plans/commands/DropUserCommand.java | 73 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../account_p0/test_nereids_account.groovy | 44 +++++++++++ 7 files changed, 136 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropUserCommand.java create mode 100644 regression-test/suites/account_p0/test_nereids_account.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 0a784515dca0d14..7e727a7f0419617 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -198,6 +198,7 @@ supportedDropStatement : DROP CATALOG RECYCLE BIN WHERE idType=STRING_LITERAL EQ id=INTEGER_VALUE #dropCatalogRecycleBin | DROP ROLE (IF EXISTS)? name=identifier #dropRole | DROP SQL_BLOCK_RULE (IF EXISTS)? identifierSeq #dropSqlBlockRule + | DROP USER (IF EXISTS)? userIdentify #dropUser ; supportedShowStatement @@ -660,7 +661,6 @@ unsupportedDropStatement | DROP (GLOBAL | SESSION | LOCAL)? FUNCTION (IF EXISTS)? functionIdentifier LEFT_PAREN functionArguments? RIGHT_PAREN #dropFunction | DROP TABLE (IF EXISTS)? name=multipartIdentifier FORCE? #dropTable - | DROP USER (IF EXISTS)? userIdentify #dropUser | DROP VIEW (IF EXISTS)? name=multipartIdentifier #dropView | DROP REPOSITORY name=identifier #dropRepository | DROP FILE name=STRING_LITERAL diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java index 77a2d791bb1eb64..f1277a1cd0a3b56 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java @@ -554,6 +554,10 @@ private void createUserInternal(UserIdentity userIdent, String roleName, byte[] } } + public void dropUser(UserIdentity userIdent, boolean ignoreIfNonExists) throws DdlException { + dropUserInternal(userIdent, ignoreIfNonExists, false); + } + // drop user public void dropUser(DropUserStmt stmt) throws DdlException { dropUserInternal(stmt.getUserIdentity(), stmt.isSetIfExists(), false); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 7c2f5f0c07020c9..cd9279c90230486 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -103,6 +103,7 @@ import org.apache.doris.nereids.DorisParser.DropProcedureContext; import org.apache.doris.nereids.DorisParser.DropRoleContext; import org.apache.doris.nereids.DorisParser.DropSqlBlockRuleContext; +import org.apache.doris.nereids.DorisParser.DropUserContext; import org.apache.doris.nereids.DorisParser.ElementAtContext; import org.apache.doris.nereids.DorisParser.ExceptContext; import org.apache.doris.nereids.DorisParser.ExceptOrReplaceContext; @@ -449,6 +450,7 @@ import org.apache.doris.nereids.trees.plans.commands.DropProcedureCommand; import org.apache.doris.nereids.trees.plans.commands.DropRoleCommand; import org.apache.doris.nereids.trees.plans.commands.DropSqlBlockRuleCommand; +import org.apache.doris.nereids.trees.plans.commands.DropUserCommand; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; import org.apache.doris.nereids.trees.plans.commands.ExportCommand; @@ -4351,6 +4353,12 @@ public LogicalPlan visitDropSqlBlockRule(DropSqlBlockRuleContext ctx) { return new DropSqlBlockRuleCommand(visitIdentifierSeq(ctx.identifierSeq()), ctx.EXISTS() != null); } + @Override + public LogicalPlan visitDropUser(DropUserContext ctx) { + UserIdentity userIdent = visitUserIdentify(ctx.userIdentify()); + return new DropUserCommand(userIdent, ctx.EXISTS() != null); + } + @Override public LogicalPlan visitShowTableId(ShowTableIdContext ctx) { long tableId = -1; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index f62e8ef77955af7..b4a9aaa51f12218 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -183,6 +183,7 @@ public enum PlanType { PREPARED_COMMAND, EXECUTE_COMMAND, DROP_SQL_BLOCK_RULE_COMMAND, + DROP_USER_COMMAND, SHOW_BACKENDS_COMMAND, SHOW_BLOCK_RULE_COMMAND, SHOW_BROKER_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropUserCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropUserCommand.java new file mode 100644 index 000000000000000..094ce06c36143e5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropUserCommand.java @@ -0,0 +1,73 @@ +// 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.nereids.trees.plans.commands; + +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.authenticate.AuthenticateType; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +/** + * drop user command + */ +public class DropUserCommand extends DropCommand { + private final boolean ifExists; + private final UserIdentity userIdent; + + /** + * constructor + */ + public DropUserCommand(UserIdentity userIdent, boolean ifExists) { + super(PlanType.DROP_USER_COMMAND); + this.userIdent = userIdent; + this.ifExists = ifExists; + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + if (Config.access_controller_type.equalsIgnoreCase("ranger-doris") + && AuthenticateType.getAuthTypeConfig() == AuthenticateType.LDAP) { + throw new AnalysisException("Drop user is prohibited when Ranger and LDAP are enabled at same time."); + } + + userIdent.analyze(); + + if (userIdent.isRootUser()) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR, "Can not drop root user"); + } + + // only user with GLOBAL level's GRANT_PRIV can drop user. + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.GRANT)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "DROP USER"); + } + Env.getCurrentEnv().getAuth().dropUser(userIdent, ifExists); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitDropUserCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 4cfa75a8efed802..0a7446075fa68cf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -43,6 +43,7 @@ import org.apache.doris.nereids.trees.plans.commands.DropProcedureCommand; import org.apache.doris.nereids.trees.plans.commands.DropRoleCommand; import org.apache.doris.nereids.trees.plans.commands.DropSqlBlockRuleCommand; +import org.apache.doris.nereids.trees.plans.commands.DropUserCommand; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand; import org.apache.doris.nereids.trees.plans.commands.ExportCommand; import org.apache.doris.nereids.trees.plans.commands.LoadCommand; @@ -409,6 +410,10 @@ default R visitDropSqlBlockRuleCommand(DropSqlBlockRuleCommand dropSqlBlockRuleC return visitCommand(dropSqlBlockRuleCommand, context); } + default R visitDropUserCommand(DropUserCommand dropUserCommand, C context) { + return visitCommand(dropUserCommand, context); + } + default R visitShowTableIdCommand(ShowTableIdCommand showTableIdCommand, C context) { return visitCommand(showTableIdCommand, context); } diff --git a/regression-test/suites/account_p0/test_nereids_account.groovy b/regression-test/suites/account_p0/test_nereids_account.groovy new file mode 100644 index 000000000000000..6c94d655662b819 --- /dev/null +++ b/regression-test/suites/account_p0/test_nereids_account.groovy @@ -0,0 +1,44 @@ +// 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. + +import org.junit.Assert; + +suite("test_nereids_account") { + // test comment + def user = "ttest_nereids_account_comment_user"; + sql """drop user if exists ${user}""" + // create user with comment + sql """create user ${user} comment 'ttest_nereids_account_comment_user_comment_create'""" + def user_create = sql "show grants for ${user}" + logger.info("user_create: " + user_create.toString()) + assertTrue(user_create.toString().contains("ttest_nereids_account_comment_user_comment_create")) + // alter user comment + sql """alter user ${user} comment 'ttest_nereids_account_comment_user_comment_alter'""" + def user_alter = sql "show grants for ${user}" + logger.info("user_alter: " + user_alter.toString()) + assertTrue(user_alter.toString().contains("ttest_nereids_account_comment_user_comment_alter")) + // drop user + checkNereidsExecute("""drop user ${user}""") + checkNereidsExecute("""drop user if exists ${user}""") + try { + sql "show grants for ${user}" + fail() + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains('not exist')) + } +} From 65e99a8a5c3aad74810111966a0ef94ee6862521 Mon Sep 17 00:00:00 2001 From: zclllhhjj Date: Fri, 22 Nov 2024 17:08:52 +0800 Subject: [PATCH 049/110] Revert "[refine](column) Remove some unnecessary get_int functions." (#44466) Reverts apache/doris#44017 In this pr, for some code path that col2's type is not FromType2 we assert it's wrong and throw error. but in fact because of some misuse in our code, many date function didn't specify the FromType2. so the "wrong" path is useful now. Only when we fixed all the calling with wrong FromType2, this pr could be applied. --- .../functions/array/function_array_slice.h | 5 +-- .../functions/array/function_array_utils.cpp | 6 +-- .../functions/array/function_array_utils.h | 2 +- .../function_date_or_datetime_computation.h | 42 +++++++++++++++---- be/src/vec/sink/vtablet_block_convertor.cpp | 4 +- 5 files changed, 40 insertions(+), 19 deletions(-) diff --git a/be/src/vec/functions/array/function_array_slice.h b/be/src/vec/functions/array/function_array_slice.h index b65ccd3de003587..2acd1d3fbe1fd43 100644 --- a/be/src/vec/functions/array/function_array_slice.h +++ b/be/src/vec/functions/array/function_array_slice.h @@ -75,11 +75,9 @@ class FunctionArraySlice : public IFunction { auto offset_column = block.get_by_position(arguments[1]).column->convert_to_full_column_if_const(); ColumnPtr length_column = nullptr; - const ColumnInt64* length_column_int64 = nullptr; if (arguments.size() > 2) { length_column = block.get_by_position(arguments[2]).column->convert_to_full_column_if_const(); - length_column_int64 = assert_cast(length_column.get()); } // extract src array column ColumnArrayExecutionData src; @@ -94,8 +92,7 @@ class FunctionArraySlice : public IFunction { ColumnArrayMutableData dst = create_mutable_data(src.nested_col, is_nullable); dst.offsets_ptr->reserve(input_rows_count); // execute - const auto* offset_column_int64 = assert_cast(offset_column.get()); - slice_array(dst, src, *offset_column_int64, length_column_int64); + slice_array(dst, src, *offset_column, length_column.get()); ColumnPtr res_column = assemble_column_array(dst); block.replace_by_position(result, std::move(res_column)); return Status::OK(); diff --git a/be/src/vec/functions/array/function_array_utils.cpp b/be/src/vec/functions/array/function_array_utils.cpp index c0141bfc8d14fc9..ab999aa21ccdd1d 100644 --- a/be/src/vec/functions/array/function_array_utils.cpp +++ b/be/src/vec/functions/array/function_array_utils.cpp @@ -78,12 +78,12 @@ MutableColumnPtr assemble_column_array(ColumnArrayMutableData& data) { } void slice_array(ColumnArrayMutableData& dst, ColumnArrayExecutionData& src, - const ColumnInt64& offset_column, const ColumnInt64* length_column) { + const IColumn& offset_column, const IColumn* length_column) { size_t cur = 0; for (size_t row = 0; row < src.offsets_ptr->size(); ++row) { size_t off = (*src.offsets_ptr)[row - 1]; size_t len = (*src.offsets_ptr)[row] - off; - Int64 start = offset_column.get_element(row); + Int64 start = offset_column.get_int(row); if (len == 0 || start == 0) { dst.offsets_ptr->push_back(cur); continue; @@ -98,7 +98,7 @@ void slice_array(ColumnArrayMutableData& dst, ColumnArrayExecutionData& src, } Int64 end; if (length_column) { - Int64 size = length_column->get_element(row); + Int64 size = length_column->get_int(row); end = std::max((Int64)off, std::min((Int64)(off + len), start + size)); } else { end = off + len; diff --git a/be/src/vec/functions/array/function_array_utils.h b/be/src/vec/functions/array/function_array_utils.h index 938bfce08fade4b..36bf811b770899c 100644 --- a/be/src/vec/functions/array/function_array_utils.h +++ b/be/src/vec/functions/array/function_array_utils.h @@ -91,7 +91,7 @@ MutableColumnPtr assemble_column_array(ColumnArrayMutableData& data); // array[offset:length] void slice_array(ColumnArrayMutableData& dst, ColumnArrayExecutionData& src, - const ColumnInt64& offset_column, const ColumnInt64* length_column); + const IColumn& offset_column, const IColumn* length_column); using ColumnArrayExecutionDatas = std::vector; } // namespace doris::vectorized diff --git a/be/src/vec/functions/function_date_or_datetime_computation.h b/be/src/vec/functions/function_date_or_datetime_computation.h index 29b8a9c9a45fdf0..90221e66c213a9c 100644 --- a/be/src/vec/functions/function_date_or_datetime_computation.h +++ b/be/src/vec/functions/function_date_or_datetime_computation.h @@ -479,6 +479,34 @@ struct DateTimeOp { } } + // use for (const DateTime, ColumnNumber) -> other_type + static void constant_vector(const FromType1& from, PaddedPODArray& vec_to, + NullMap& null_map, const IColumn& delta) { + size_t size = delta.size(); + vec_to.resize(size); + null_map.resize_fill(size, false); + + for (size_t i = 0; i < size; ++i) { + vec_to[i] = Transform::execute(from, delta.get_int(i), + reinterpret_cast(null_map[i])); + } + } + static void constant_vector(const FromType1& from, PaddedPODArray& vec_to, + const IColumn& delta) { + size_t size = delta.size(); + vec_to.resize(size); + bool invalid = true; + + for (size_t i = 0; i < size; ++i) { + vec_to[i] = Transform::execute(from, delta.get_int(i), invalid); + + if (UNLIKELY(invalid)) { + throw Exception(ErrorCode::OUT_OF_BOUND, "Operation {} {} {} out of range", + Transform::name, from, delta.get_int(i)); + } + } + } + static void constant_vector(const FromType1& from, PaddedPODArray& vec_to, NullMap& null_map, const PaddedPODArray& delta) { size_t size = delta.size(); @@ -619,10 +647,9 @@ struct DateTimeAddIntervalImpl { col_to->get_data(), null_map->get_data(), delta_vec_column->get_data()); } else { - return Status::RuntimeError( - "Illegal column {} of first argument of function {}", - block.get_by_position(arguments[0]).column->get_name(), - Transform::name); + Op::constant_vector(sources_const->template get_value(), + col_to->get_data(), null_map->get_data(), + *not_nullable_column_ptr_arg1); } if (const auto* nullable_col = check_and_get_column( block.get_by_position(arguments[0]).column.get())) { @@ -650,10 +677,9 @@ struct DateTimeAddIntervalImpl { Op::constant_vector(sources_const->template get_value(), col_to->get_data(), delta_vec_column->get_data()); } else { - return Status::RuntimeError( - "Illegal column {} of first argument of function {}", - block.get_by_position(arguments[0]).column->get_name(), - Transform::name); + Op::constant_vector(sources_const->template get_value(), + col_to->get_data(), + *block.get_by_position(arguments[1]).column); } block.replace_by_position(result, std::move(col_to)); } diff --git a/be/src/vec/sink/vtablet_block_convertor.cpp b/be/src/vec/sink/vtablet_block_convertor.cpp index e9b8be6f922156c..26de6ea6c7e3d18 100644 --- a/be/src/vec/sink/vtablet_block_convertor.cpp +++ b/be/src/vec/sink/vtablet_block_convertor.cpp @@ -45,7 +45,6 @@ #include "vec/columns/column_nullable.h" #include "vec/columns/column_string.h" #include "vec/columns/column_struct.h" -#include "vec/columns/columns_number.h" #include "vec/common/assert_cast.h" #include "vec/core/block.h" #include "vec/core/types.h" @@ -533,7 +532,6 @@ Status OlapTableBlockConvertor::_fill_auto_inc_cols(vectorized::Block* block, si } else if (const auto* src_nullable_column = check_and_get_column(src_column_ptr)) { auto src_nested_column_ptr = src_nullable_column->get_nested_column_ptr(); - const auto* src_int_64 = assert_cast(src_nested_column_ptr.get()); const auto& null_map_data = src_nullable_column->get_null_map_data(); dst_values.reserve(rows); for (size_t i = 0; i < rows; i++) { @@ -547,7 +545,7 @@ Status OlapTableBlockConvertor::_fill_auto_inc_cols(vectorized::Block* block, si for (size_t i = 0; i < rows; i++) { dst_values.emplace_back((null_map_data[i] != 0) ? _auto_inc_id_allocator.next_id() - : src_int_64->get_element(i)); + : src_nested_column_ptr->get_int(i)); } } else { return Status::OK(); From 5b20207d1eb9ca7ed1fb3669e3621bbf1e166c6c Mon Sep 17 00:00:00 2001 From: walter Date: Fri, 22 Nov 2024 17:57:29 +0800 Subject: [PATCH 050/110] [fix](suites) Fix syncer ingest binlog with multiple replicas (#44444) --- .../doris/regression/suite/Syncer.groovy | 95 +++++++++++-------- .../regression/suite/SyncerContext.groovy | 27 +++++- .../test_ingest_binlog.groovy | 36 +++---- 3 files changed, 98 insertions(+), 60 deletions(-) diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Syncer.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Syncer.groovy index 894b42824ef7074..2195e7e745afd7b 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Syncer.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Syncer.groovy @@ -700,8 +700,13 @@ class Syncer { // step 3.2: get partition/indexId/tabletId partitionSQl += "/" + meta.indexId.toString() sqlInfo = sendSql.call(partitionSQl, toSrc) + Map replicaMap = Maps.newHashMap() for (List row : sqlInfo) { - meta.tabletMeta.put(row[0] as Long, row[2] as Long) + Long tabletId = row[0] as Long + if (!meta.tabletMeta.containsKey(tabletId)) { + meta.tabletMeta.put(tabletId, new TabletMeta()) + } + meta.tabletMeta[tabletId].replicas.put(row[1] as Long, row[2] as Long) } if (meta.tabletMeta.isEmpty()) { logger.error("Target cluster get (partitionId/indexId)-(${info.key}/${meta.indexId}) tabletIds fault.") @@ -816,49 +821,57 @@ class Syncer { while (srcTabletIter.hasNext()) { Entry srcTabletMap = srcTabletIter.next() Entry tarTabletMap = tarTabletIter.next() + TabletMeta srcTabletMeta = srcTabletMap.value + TabletMeta tarTabletMeta = tarTabletMap.value + + Iterator srcReplicaIter = srcTabletMeta.replicas.iterator() + Iterator tarReplicaIter = tarTabletMeta.replicas.iterator() + while (srcReplicaIter.hasNext()) { + Entry srcReplicaMap = srcReplicaIter.next() + Entry tarReplicaMap = tarReplicaIter.next() + BackendClientImpl srcClient = context.sourceBackendClients.get(srcReplicaMap.value) + if (srcClient == null) { + logger.error("Can't find src tabletId-${srcReplicaMap.key} -> beId-${srcReplicaMap.value}") + return false + } + BackendClientImpl tarClient = context.targetBackendClients.get(tarReplicaMap.value) + if (tarClient == null) { + logger.error("Can't find target tabletId-${tarReplicaMap.key} -> beId-${tarReplicaMap.value}") + return false + } - BackendClientImpl srcClient = context.sourceBackendClients.get(srcTabletMap.value) - if (srcClient == null) { - logger.error("Can't find src tabletId-${srcTabletMap.key} -> beId-${srcTabletMap.value}") - return false - } - BackendClientImpl tarClient = context.targetBackendClients.get(tarTabletMap.value) - if (tarClient == null) { - logger.error("Can't find target tabletId-${tarTabletMap.key} -> beId-${tarTabletMap.value}") - return false - } - - tarPartition.value.version = srcPartition.value.version - long partitionId = fakePartitionId == -1 ? tarPartition.key : fakePartitionId - long version = fakeVersion == -1 ? partitionRecord.version : fakeVersion - - TIngestBinlogRequest request = new TIngestBinlogRequest() - TUniqueId uid = new TUniqueId(-1, -1) - request.setTxnId(txnId) - request.setRemoteTabletId(srcTabletMap.key) - request.setBinlogVersion(version) - request.setRemoteHost(srcClient.address.hostname) - request.setRemotePort(srcClient.httpPort.toString()) - request.setPartitionId(partitionId) - request.setLocalTabletId(tarTabletMap.key) - request.setLoadId(uid) - logger.info("request -> ${request}") - TIngestBinlogResult result = tarClient.client.ingestBinlog(request) - if (!checkIngestBinlog(result)) { - logger.error("Ingest binlog error! result: ${result}") - return false - } + tarPartition.value.version = srcPartition.value.version + long partitionId = fakePartitionId == -1 ? tarPartition.key : fakePartitionId + long version = fakeVersion == -1 ? partitionRecord.version : fakeVersion + + TIngestBinlogRequest request = new TIngestBinlogRequest() + TUniqueId uid = new TUniqueId(-1, -1) + request.setTxnId(txnId) + request.setRemoteTabletId(srcTabletMap.key) + request.setBinlogVersion(version) + request.setRemoteHost(srcClient.address.hostname) + request.setRemotePort(srcClient.httpPort.toString()) + request.setPartitionId(partitionId) + request.setLocalTabletId(tarTabletMap.key) + request.setLoadId(uid) + logger.info("request -> ${request}") + TIngestBinlogResult result = tarClient.client.ingestBinlog(request) + if (!checkIngestBinlog(result)) { + logger.error("Ingest binlog error! result: ${result}") + return false + } - if (context.txnInsert) { - List tabletCommitInfos = subTxnIdToTabletCommitInfos.get(txnId) - if (tabletCommitInfos == null) { - tabletCommitInfos = new ArrayList() - subTxnIdToTabletCommitInfos.put(txnId, tabletCommitInfos) - subTxnIdToTableId.put(txnId, tarTableMeta.id) + if (context.txnInsert) { + List tabletCommitInfos = subTxnIdToTabletCommitInfos.get(txnId) + if (tabletCommitInfos == null) { + tabletCommitInfos = new ArrayList() + subTxnIdToTabletCommitInfos.put(txnId, tabletCommitInfos) + subTxnIdToTableId.put(txnId, tarTableMeta.id) + } + tabletCommitInfos.add(new TTabletCommitInfo(tarTabletMap.key, tarReplicaMap.value)) + } else { + addCommitInfo(tarTabletMap.key, tarReplicaMap.value) } - tabletCommitInfos.add(new TTabletCommitInfo(tarTabletMap.key, tarTabletMap.value)) - } else { - addCommitInfo(tarTabletMap.key, tarTabletMap.value) } } } diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SyncerContext.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SyncerContext.groovy index b86f012aa877589..3202db4011f8715 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SyncerContext.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SyncerContext.groovy @@ -32,10 +32,22 @@ import groovy.util.logging.Slf4j import java.sql.Connection +class TabletMeta { + public TreeMap replicas + + TabletMeta() { + this.replicas = new TreeMap() + } + + String toString() { + return "TabletMeta: { replicas: " + replicas.toString() + " }" + } +} + class PartitionMeta { public long version public long indexId - public TreeMap tabletMeta + public TreeMap tabletMeta PartitionMeta(long indexId, long version) { this.indexId = indexId @@ -219,6 +231,19 @@ class SyncerContext { } else if (srcTabletMeta.size() != tarTabletMeta.size()) { return false } + + Iterator srcTabletIter = srcTabletMeta.iterator() + Iterator tarTabletIter = tarTabletMeta.iterator() + while (srcTabletIter.hasNext()) { + Map srcReplicaMap = srcTabletIter.next().value.replicas + Map tarReplicaMap = tarTabletIter.next().value.replicas + + if (srcReplicaMap.isEmpty() || tarReplicaMap.isEmpty()) { + return false + } else if (srcReplicaMap.size() != tarReplicaMap.size()) { + return false + } + } } }) diff --git a/regression-test/suites/ccr_mow_syncer_p0/test_ingest_binlog.groovy b/regression-test/suites/ccr_mow_syncer_p0/test_ingest_binlog.groovy index 233d57aa7b3342e..e07529718ee467e 100644 --- a/regression-test/suites/ccr_mow_syncer_p0/test_ingest_binlog.groovy +++ b/regression-test/suites/ccr_mow_syncer_p0/test_ingest_binlog.groovy @@ -27,16 +27,16 @@ suite("test_mow_ingest_binlog") { def test_num = 0 sql "DROP TABLE IF EXISTS ${tableName}" sql """ - CREATE TABLE if NOT EXISTS ${tableName} + CREATE TABLE if NOT EXISTS ${tableName} ( `test` INT, `id` INT ) ENGINE=OLAP UNIQUE KEY(`test`, `id`) - DISTRIBUTED BY HASH(id) BUCKETS 1 - PROPERTIES ( - "enable_unique_key_merge_on_write" = "true", + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ( + "enable_unique_key_merge_on_write" = "true", "replication_allocation" = "tag.location.default: 1" ) """ @@ -44,19 +44,19 @@ suite("test_mow_ingest_binlog") { target_sql "DROP TABLE IF EXISTS ${tableName}" target_sql """ - CREATE TABLE if NOT EXISTS ${tableName} - ( - `test` INT, - `id` INT - ) - ENGINE=OLAP - UNIQUE KEY(`test`, `id`) - DISTRIBUTED BY HASH(id) BUCKETS 1 - PROPERTIES ( - "enable_unique_key_merge_on_write" = "true", - "replication_allocation" = "tag.location.default: 1" - ) - """ + CREATE TABLE if NOT EXISTS ${tableName} + ( + `test` INT, + `id` INT + ) + ENGINE=OLAP + UNIQUE KEY(`test`, `id`) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ( + "enable_unique_key_merge_on_write" = "true", + "replication_allocation" = "tag.location.default: 1" + ) + """ assertTrue(syncer.getTargetMeta("${tableName}")) @@ -124,4 +124,4 @@ suite("test_mow_ingest_binlog") { // End Test 2 syncer.closeBackendClients() -} \ No newline at end of file +} From 55db8ca1cc5e1662e865605a9f86f8e5a1e19626 Mon Sep 17 00:00:00 2001 From: meiyi Date: Fri, 22 Nov 2024 19:42:47 +0800 Subject: [PATCH 051/110] [fix](regression) fix txn insert case (#44395) --- regression-test/suites/insert_p0/insert_with_null.groovy | 5 +++++ .../suites/insert_p0/transaction/txn_insert.groovy | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/regression-test/suites/insert_p0/insert_with_null.groovy b/regression-test/suites/insert_p0/insert_with_null.groovy index 19e49a749cf127f..b89d642e33ba15e 100644 --- a/regression-test/suites/insert_p0/insert_with_null.groovy +++ b/regression-test/suites/insert_p0/insert_with_null.groovy @@ -35,6 +35,11 @@ suite("insert_with_null") { ); """ + sql "ADMIN SET FRONTEND CONFIG ('commit_timeout_second' = '100')" + onFinish { + sql "ADMIN SET FRONTEND CONFIG ('commit_timeout_second' = '30')" + } + def getRowCount = { expectedRowCount -> def retry = 0 while (retry < 30) { diff --git a/regression-test/suites/insert_p0/transaction/txn_insert.groovy b/regression-test/suites/insert_p0/transaction/txn_insert.groovy index 9459297a00b3251..6653c05740e3b19 100644 --- a/regression-test/suites/insert_p0/transaction/txn_insert.groovy +++ b/regression-test/suites/insert_p0/transaction/txn_insert.groovy @@ -42,6 +42,11 @@ suite("txn_insert") { return null } + sql "ADMIN SET FRONTEND CONFIG ('commit_timeout_second' = '100')" + onFinish { + sql "ADMIN SET FRONTEND CONFIG ('commit_timeout_second' = '30')" + } + for (def use_nereids_planner : [/*false,*/ true]) { sql " SET enable_nereids_planner = $use_nereids_planner; " From 6a31e0c86e3ebab3fb6bbd10292784f19674d425 Mon Sep 17 00:00:00 2001 From: meiyi Date: Fri, 22 Nov 2024 19:46:28 +0800 Subject: [PATCH 052/110] [fix](cluster key) fix cluster key update (#44413) for mow with cluster key, update may get: ``` mysql> update t2_cluster_key set b = 22 where c = 2; ERROR 1105 (HY000): errCode = 2, detailMessage = Column has no default value, column=c ``` --- .../trees/plans/commands/UpdateCommand.java | 3 +- .../test_mix_partial_update.out | 36 ++++++++-------- .../test_partial_update_with_update_stmt.out | 12 +++--- .../data/unique_with_mow_c_p0/test_update.out | 6 +++ ...est_partial_update_with_update_stmt.groovy | 18 +++----- .../unique_with_mow_c_p0/test_update.groovy | 43 +++++++++++++++++++ 6 files changed, 82 insertions(+), 36 deletions(-) create mode 100644 regression-test/data/unique_with_mow_c_p0/test_update.out create mode 100644 regression-test/suites/unique_with_mow_c_p0/test_update.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java index 51cfbf0b27271ff..ada4ca9035c47aa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java @@ -161,7 +161,8 @@ public LogicalPlan completeQueryPlan(ConnectContext ctx, LogicalPlan logicalQuer boolean isPartialUpdate = targetTable.getEnableUniqueKeyMergeOnWrite() && selectItems.size() < targetTable.getColumns().size() && targetTable.getSequenceCol() == null - && partialUpdateColNameToExpression.size() <= targetTable.getFullSchema().size() * 3 / 10; + && partialUpdateColNameToExpression.size() <= targetTable.getFullSchema().size() * 3 / 10 + && !targetTable.isUniqKeyMergeOnWriteWithClusterKeys(); List partialUpdateColNames = new ArrayList<>(); List partialUpdateSelectItems = new ArrayList<>(); diff --git a/regression-test/data/unique_with_mow_c_p0/partial_update/test_mix_partial_update.out b/regression-test/data/unique_with_mow_c_p0/partial_update/test_mix_partial_update.out index cb5c64e0d8199af..ce5681114696403 100644 --- a/regression-test/data/unique_with_mow_c_p0/partial_update/test_mix_partial_update.out +++ b/regression-test/data/unique_with_mow_c_p0/partial_update/test_mix_partial_update.out @@ -97,8 +97,8 @@ 2 -- !select_C -- -1 \N \N \N -2 \N \N \N +1 1 1 {"a":100, "b":100} +2 2 2 {"a":200, "b":200} -- !select_CC -- 1 @@ -107,8 +107,8 @@ 2 -- !select_D -- -1 \N \N \N -2 \N \N \N +1 1 1 {"a":100, "b":100} +2 2 2 {"a":200, "b":200} 3 3 3 {"a":300, "b":300} -- !select_DD -- @@ -118,8 +118,8 @@ 1 -- !select_E -- -1 \N \N \N -2 \N \N \N +1 1 1 {"a":100, "b":100} +2 2 2 {"a":200, "b":200} 3 3 3 {"a":300, "b":300} 4 4 4 {"a":400, "b":400} @@ -148,7 +148,7 @@ 1 -- !select_C -- -1 \N \N \N +1 1 1 {"a":100, "b":100} -- !select_CC -- 1 @@ -157,7 +157,7 @@ 1 -- !select_D -- -1 \N \N \N +1 1 1 {"a":100, "b":100} 3 3 3 {"a":300, "b":300} -- !select_DD -- @@ -167,7 +167,7 @@ 1 -- !select_E -- -1 \N \N \N +1 1 1 {"a":100, "b":100} 3 3 3 {"a":300, "b":300} 4 4 4 {"a":400, "b":400} @@ -275,8 +275,8 @@ 2 -- !select_C -- -1 \N \N \N -2 \N \N \N +1 1 1 {"a":100, "b":100} +2 2 2 {"a":200, "b":200} -- !select_CC -- 1 @@ -285,8 +285,8 @@ 2 -- !select_D -- -1 \N \N \N -2 \N \N \N +1 1 1 {"a":100, "b":100} +2 2 2 {"a":200, "b":200} 3 3 3 {"a":300, "b":300} -- !select_DD -- @@ -296,8 +296,8 @@ 1 -- !select_E -- -1 \N \N \N -2 \N \N \N +1 1 1 {"a":100, "b":100} +2 2 2 {"a":200, "b":200} 3 3 3 {"a":300, "b":300} 4 4 4 {"a":400, "b":400} @@ -326,7 +326,7 @@ 1 -- !select_C -- -1 \N \N \N +1 1 1 {"a":100, "b":100} -- !select_CC -- 1 @@ -335,7 +335,7 @@ 1 -- !select_D -- -1 \N \N \N +1 1 1 {"a":100, "b":100} 3 3 3 {"a":300, "b":300} -- !select_DD -- @@ -345,7 +345,7 @@ 1 -- !select_E -- -1 \N \N \N +1 1 1 {"a":100, "b":100} 3 3 3 {"a":300, "b":300} 4 4 4 {"a":400, "b":400} diff --git a/regression-test/data/unique_with_mow_c_p0/partial_update/test_partial_update_with_update_stmt.out b/regression-test/data/unique_with_mow_c_p0/partial_update/test_partial_update_with_update_stmt.out index 6e06f6aeeabca23..43c2e32104aec2c 100644 --- a/regression-test/data/unique_with_mow_c_p0/partial_update/test_partial_update_with_update_stmt.out +++ b/regression-test/data/unique_with_mow_c_p0/partial_update/test_partial_update_with_update_stmt.out @@ -1,17 +1,17 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select_default -- -1 doris 1000 123 1 +1 doris 4000 123 1 2 doris2 2000 223 1 -- !select_join -- -1 doris 1000 123 1 -2 doris4 8000 \N 4321 +1 doris 4000 123 1 +2 doris4 8000 223 1 -- !select_default -- -1 doris 1000 123 1 +1 doris 4000 123 1 2 doris2 2000 223 1 -- !select_join -- -1 doris 1000 123 1 -2 doris4 8000 \N 4321 +1 doris 4000 123 1 +2 doris4 8000 223 1 diff --git a/regression-test/data/unique_with_mow_c_p0/test_update.out b/regression-test/data/unique_with_mow_c_p0/test_update.out new file mode 100644 index 000000000000000..d1eedc98b1e3be8 --- /dev/null +++ b/regression-test/data/unique_with_mow_c_p0/test_update.out @@ -0,0 +1,6 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +10 200 3 +11 200 3 +12 22 32 + diff --git a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_with_update_stmt.groovy b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_with_update_stmt.groovy index 88d6d1ccc46d5fb..0a29b382b306aea 100644 --- a/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_with_update_stmt.groovy +++ b/regression-test/suites/unique_with_mow_c_p0/partial_update/test_partial_update_with_update_stmt.groovy @@ -57,10 +57,9 @@ suite("test_primary_key_partial_update_with_update_stmt", "p0") { """ // case 1: partially update normally - test { - sql """ update ${tableName} set score = 4000 where id = 1 """ - exception "Column has no default value" - } + sql """ + update ${tableName} set score = 4000 where id = 1 + """ sql "sync" @@ -69,15 +68,12 @@ suite("test_primary_key_partial_update_with_update_stmt", "p0") { """ // case 2: partially update non-exist key - test { - def result1 = sql """ + def result1 = sql """ update ${tableName} set score = 2000 where id = 3 """ - /*assertTrue(result1.size() == 1) - assertTrue(result1[0].size() == 1) - assertTrue(result1[0][0] == 0, "Query OK, 0 rows affected")*/ - exception "Column has no default value" - } + assertTrue(result1.size() == 1) + assertTrue(result1[0].size() == 1) + assertTrue(result1[0][0] == 0, "Query OK, 0 rows affected") sql "sync" diff --git a/regression-test/suites/unique_with_mow_c_p0/test_update.groovy b/regression-test/suites/unique_with_mow_c_p0/test_update.groovy new file mode 100644 index 000000000000000..b062da545167083 --- /dev/null +++ b/regression-test/suites/unique_with_mow_c_p0/test_update.groovy @@ -0,0 +1,43 @@ +// 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_update") { + def tableName = "test_update" + sql """ DROP TABLE IF EXISTS ${tableName} """ + onFinish { + try_sql("DROP TABLE IF EXISTS ${tableName}") + } + + sql """ + CREATE TABLE `$tableName` ( + `a` int NOT NULL, + `b` int NOT NULL, + `c` int NOT NULL + ) ENGINE=OLAP + UNIQUE KEY(`a`) + CLUSTER BY (`b`) + DISTRIBUTED BY HASH(`a`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1", + "enable_unique_key_merge_on_write" = "true" + ); + """ + + sql """ insert into $tableName values(10, 20, 3), (11, 21, 3), (12, 22, 32); """ + sql """ update $tableName set b = 200 where c = 3; """ + order_qt_sql "select * from $tableName" +} From 2753b709c42448f015f5e2d72185d740ec46f9b9 Mon Sep 17 00:00:00 2001 From: meiyi Date: Fri, 22 Nov 2024 19:48:06 +0800 Subject: [PATCH 053/110] [fix](cluster key) support fuzzy random_add_cluster_keys_for_mow config (#44485) support fuzzy set fe config for `random_add_cluster_keys_for_mow` by add: ``` use_fuzzy_conf=true fuzzy_test_type=daily or rqg ``` --- .../main/java/org/apache/doris/common/Config.java | 8 +++++++- .../src/main/java/org/apache/doris/DorisFE.java | 13 +++++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 689c85940dfc394..be0390db584ca92 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -2180,7 +2180,7 @@ public class Config extends ConfigBase { * only for certain test type. E.g. only settting batch_size to small * value for p0. */ - @ConfField(mutable = true, masterOnly = false, options = {"p0"}) + @ConfField(mutable = true, masterOnly = false, options = {"p0", "daily", "rqg"}) public static String fuzzy_test_type = ""; /** @@ -2189,6 +2189,12 @@ public class Config extends ConfigBase { @ConfField(mutable = true, masterOnly = false) public static boolean use_fuzzy_session_variable = false; + /** + * Set config variables randomly to check more issues in github workflow + */ + @ConfField(mutable = true, masterOnly = false) + public static boolean use_fuzzy_conf = false; + /** * Max num of same name meta informatntion in catalog recycle bin. * Default is 3. diff --git a/fe/fe-core/src/main/java/org/apache/doris/DorisFE.java b/fe/fe-core/src/main/java/org/apache/doris/DorisFE.java index d5b7cd7354eb1fa..d3539e96f8162b4 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/DorisFE.java +++ b/fe/fe-core/src/main/java/org/apache/doris/DorisFE.java @@ -61,6 +61,7 @@ import java.nio.channels.FileLock; import java.nio.channels.OverlappingFileLockException; import java.nio.file.StandardOpenOption; +import java.time.LocalDate; import java.util.concurrent.TimeUnit; public class DorisFE { @@ -162,6 +163,8 @@ public static void start(String dorisHomeDir, String pidDir, String[] args, Star System.exit(-1); } + fuzzyConfigs(); + LOG.info("Doris FE starting..."); FrontendOptions.init(); @@ -529,6 +532,16 @@ public static void overwriteConfigs() { } } + private static void fuzzyConfigs() { + if (!Config.use_fuzzy_conf) { + return; + } + if (Config.fuzzy_test_type.equalsIgnoreCase("daily") || Config.fuzzy_test_type.equalsIgnoreCase("rqg")) { + Config.random_add_cluster_keys_for_mow = (LocalDate.now().getDayOfMonth() % 2 == 0); + LOG.info("fuzzy set random_add_cluster_keys_for_mow={}", Config.random_add_cluster_keys_for_mow); + } + } + public static class StartupOptions { public boolean enableHttpServer = true; public boolean enableQeService = true; From f5637a07d7874347d6fb5e4d1387c9671378f195 Mon Sep 17 00:00:00 2001 From: Luwei Date: Fri, 22 Nov 2024 20:03:30 +0800 Subject: [PATCH 054/110] [improvement](compaction) reduce tablet skip compaction time (#44273) The time for tablet skip compaction is 120 seconds, which is too long. In the scenario of high-frequency import (mow), it leads to a high compaction score. Therefore, reducing the skip time to 10 seconds is necessary. --- be/src/common/config.cpp | 6 ++++-- be/src/common/config.h | 1 + be/src/olap/tablet.cpp | 4 ++-- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index d9b9a02260a4e6b..2938e81a25eb01f 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -418,7 +418,9 @@ DEFINE_mInt64(base_compaction_max_compaction_score, "20"); DEFINE_mDouble(base_compaction_min_data_ratio, "0.3"); DEFINE_mInt64(base_compaction_dup_key_max_file_size_mbytes, "1024"); -DEFINE_Bool(enable_skip_tablet_compaction, "false"); +DEFINE_Bool(enable_skip_tablet_compaction, "true"); +DEFINE_mInt32(skip_tablet_compaction_second, "10"); + // output rowset of cumulative compaction total disk size exceed this config size, // this rowset will be given to base compaction, unit is m byte. DEFINE_mInt64(compaction_promotion_size_mbytes, "1024"); @@ -454,7 +456,7 @@ DEFINE_mInt32(multi_get_max_threads, "10"); DEFINE_mInt64(total_permits_for_compaction_score, "10000"); // sleep interval in ms after generated compaction tasks -DEFINE_mInt32(generate_compaction_tasks_interval_ms, "10"); +DEFINE_mInt32(generate_compaction_tasks_interval_ms, "100"); // sleep interval in second after update replica infos DEFINE_mInt32(update_replica_infos_interval_seconds, "60"); diff --git a/be/src/common/config.h b/be/src/common/config.h index 7f18406eeee721b..e6247f596a177cb 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -470,6 +470,7 @@ DECLARE_mDouble(base_compaction_min_data_ratio); DECLARE_mInt64(base_compaction_dup_key_max_file_size_mbytes); DECLARE_Bool(enable_skip_tablet_compaction); +DECLARE_mInt32(skip_tablet_compaction_second); // output rowset of cumulative compaction total disk size exceed this config size, // this rowset will be given to base compaction, unit is m byte. DECLARE_mInt64(compaction_promotion_size_mbytes); diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 62efef111d3584d..8a0e23e75b86adb 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -2542,10 +2542,10 @@ void Tablet::set_skip_compaction(bool skip, CompactionType compaction_type, int6 bool Tablet::should_skip_compaction(CompactionType compaction_type, int64_t now) { if (compaction_type == CompactionType::CUMULATIVE_COMPACTION && _skip_cumu_compaction && - now < _skip_cumu_compaction_ts + 120) { + now < _skip_cumu_compaction_ts + config::skip_tablet_compaction_second) { return true; } else if (compaction_type == CompactionType::BASE_COMPACTION && _skip_base_compaction && - now < _skip_base_compaction_ts + 120) { + now < _skip_base_compaction_ts + config::skip_tablet_compaction_second) { return true; } return false; From 71f5cc48cbc6c11d7d18b421b631f3a996387205 Mon Sep 17 00:00:00 2001 From: walter Date: Fri, 22 Nov 2024 20:06:03 +0800 Subject: [PATCH 055/110] [fix](index) Fix create index/index def to sql (#44392) Fix toSql for CreateIndexClause with alter --- .../doris/alter/SchemaChangeHandler.java | 4 ++-- .../doris/analysis/CreateIndexClause.java | 2 +- .../doris/analysis/DropIndexClause.java | 2 +- .../org/apache/doris/analysis/IndexDef.java | 2 +- .../java/org/apache/doris/catalog/Index.java | 2 +- .../doris/analysis/AlterTableStmtTest.java | 23 +++++++++++++++++++ .../doris/analysis/CreateIndexClauseTest.java | 12 +++++++++- .../doris/analysis/DropIndexClauseTest.java | 10 +++++++- .../apache/doris/analysis/IndexDefTest.java | 4 ++-- 9 files changed, 51 insertions(+), 10 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index 6eaf7d5522c96d2..8d257bc2d78d1f6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -1922,7 +1922,7 @@ public void process(String rawSql, List alterClauses, Database db, // index id -> index schema Map> indexSchemaMap = new HashMap<>(); - //for multi add colmuns clauses + //for multi add columns clauses //index id -> index col_unique_id supplier Map colUniqueIdSupplierMap = new HashMap<>(); for (Map.Entry> entry : olapTable.getIndexIdToSchema(true).entrySet()) { @@ -2752,7 +2752,7 @@ private boolean processAddIndex(CreateIndexClause alterClause, OlapTable olapTab // the column name in CreateIndexClause is not check case sensitivity, // when send index description to BE, there maybe cannot find column by name, // so here update column name in CreateIndexClause after checkColumn for indexDef, - // there will use the column name in olapTable insead of the column name in CreateIndexClause. + // there will use the column name in olapTable instead of the column name in CreateIndexClause. alterIndex.setColumns(indexDef.getColumns()); alterIndex.setColumnUniqueIds(indexDef.getColumnUniqueIds()); newIndexes.add(alterIndex); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateIndexClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateIndexClause.java index 86df87453ad5751..16f90432e7b7ab3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateIndexClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateIndexClause.java @@ -89,7 +89,7 @@ public boolean needChangeMTMVState() { @Override public String toSql() { if (alter) { - return indexDef.toSql(); + return "ADD " + indexDef.toSql(); } else { return "CREATE " + indexDef.toSql(tableName.toSql()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropIndexClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropIndexClause.java index f72136ee2b3f419..783bed775be0f6d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropIndexClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropIndexClause.java @@ -81,7 +81,7 @@ public boolean needChangeMTMVState() { @Override public String toSql() { StringBuilder stringBuilder = new StringBuilder(); - stringBuilder.append("DROP INDEX ").append(indexName); + stringBuilder.append("DROP INDEX ").append("`" + indexName + "`"); if (!alter) { stringBuilder.append(" ON ").append(tableName.toSql()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java index 39b32662614f153..7f406d2a7e1365c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java @@ -118,7 +118,7 @@ public String toSql() { public String toSql(String tableName) { StringBuilder sb = new StringBuilder("INDEX "); - sb.append(indexName); + sb.append("`" + indexName + "`"); if (tableName != null && !tableName.isEmpty()) { sb.append(" ON ").append(tableName); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Index.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Index.java index 40db2f1d5b01d50..8d4cc0ee4aafb2b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Index.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Index.java @@ -54,7 +54,7 @@ public class Index implements Writable { public static final int INDEX_ID_INIT_VALUE = -1; @SerializedName(value = "i", alternate = {"indexId"}) - private long indexId = -1; // -1 for compatibale + private long indexId = -1; // -1 for compatiable @SerializedName(value = "in", alternate = {"indexName"}) private String indexName; @SerializedName(value = "c", alternate = {"columns"}) diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/AlterTableStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/AlterTableStmtTest.java index 9f32556ce9c2019..9476df59cd14194 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/AlterTableStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/AlterTableStmtTest.java @@ -123,4 +123,27 @@ public void testEnableFeature() throws UserException { stmt.toSql()); Assert.assertEquals("testDb", stmt.getTbl().getDb()); } + + @Test + public void testCreateIndex() throws UserException { + List ops = Lists.newArrayList(); + ops.add(new CreateIndexClause( + new TableName(InternalCatalog.INTERNAL_CATALOG_NAME, "db", "table"), + new IndexDef("index1", false, Lists.newArrayList("col1"), IndexDef.IndexType.INVERTED, null, "balabala"), + true)); + AlterTableStmt stmt = new AlterTableStmt(new TableName(internalCtl, "testDb", "testTbl"), ops); + stmt.analyze(analyzer); + Assert.assertEquals("ALTER TABLE `testDb`.`testTbl` ADD INDEX `index1` (`col1`) USING INVERTED COMMENT 'balabala'", + stmt.toSql()); + } + + @Test + public void testDropIndex() throws UserException { + List ops = Lists.newArrayList(); + ops.add(new DropIndexClause("index1", false, + new TableName(InternalCatalog.INTERNAL_CATALOG_NAME, "db", "table"), true)); + AlterTableStmt stmt = new AlterTableStmt(new TableName(internalCtl, "testDb", "testTbl"), ops); + stmt.analyze(analyzer); + Assert.assertEquals("ALTER TABLE `testDb`.`testTbl` DROP INDEX `index1`", stmt.toSql()); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateIndexClauseTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateIndexClauseTest.java index 7341b8e7bcfd1e7..86e633d2dfebe61 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateIndexClauseTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateIndexClauseTest.java @@ -40,11 +40,21 @@ public void testNormal() throws AnalysisException { new IndexDef("index1", false, Lists.newArrayList("col1"), IndexDef.IndexType.INVERTED, null, "balabala"), false); clause.analyze(analyzer); - Assert.assertEquals("CREATE INDEX index1 ON `db`.`table` (`col1`) USING INVERTED COMMENT 'balabala'", + Assert.assertEquals("CREATE INDEX `index1` ON `db`.`table` (`col1`) USING INVERTED COMMENT 'balabala'", clause.toSql()); } + @Test + public void testAlter() throws AnalysisException { + CreateIndexClause clause = new CreateIndexClause( + new TableName(InternalCatalog.INTERNAL_CATALOG_NAME, "db", "table"), + new IndexDef("index1", false, Lists.newArrayList("col1"), IndexDef.IndexType.INVERTED, null, "balabala"), + true); + clause.analyze(analyzer); + Assert.assertEquals("ADD INDEX `index1` (`col1`) USING INVERTED COMMENT 'balabala'", clause.toSql()); + } + @Test(expected = AnalysisException.class) public void testDuplIndex() throws AnalysisException { CreateIndexClause clause = new CreateIndexClause( diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropIndexClauseTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropIndexClauseTest.java index da43a1fe563ea0a..e1659cd03cfcfa0 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropIndexClauseTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropIndexClauseTest.java @@ -39,7 +39,15 @@ public void testNormal() throws UserException { DropIndexClause clause = new DropIndexClause("index1", false, new TableName(InternalCatalog.INTERNAL_CATALOG_NAME, "db", "table"), false); clause.analyze(analyzer); - Assert.assertEquals("DROP INDEX index1 ON `db`.`table`", clause.toSql()); + Assert.assertEquals("DROP INDEX `index1` ON `db`.`table`", clause.toSql()); + } + + @Test + public void testAlter() throws UserException { + DropIndexClause clause = new DropIndexClause("index1", false, + new TableName(InternalCatalog.INTERNAL_CATALOG_NAME, "db", "table"), true); + clause.analyze(analyzer); + Assert.assertEquals("DROP INDEX `index1`", clause.toSql()); } @Test(expected = AnalysisException.class) diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/IndexDefTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/IndexDefTest.java index ccabece832d023c..56c78e2430a5ed9 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/IndexDefTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/IndexDefTest.java @@ -77,8 +77,8 @@ public void testAnalyzeExpection() throws AnalysisException { @Test public void toSql() { - Assert.assertEquals("INDEX index1 (`col1`) USING INVERTED COMMENT 'balabala'", def.toSql()); - Assert.assertEquals("INDEX index1 ON table1 (`col1`) USING INVERTED COMMENT 'balabala'", + Assert.assertEquals("INDEX `index1` (`col1`) USING INVERTED COMMENT 'balabala'", def.toSql()); + Assert.assertEquals("INDEX `index1` ON table1 (`col1`) USING INVERTED COMMENT 'balabala'", def.toSql("table1")); } } From 047b324b3e9646a47de8b67a5ddcd452a6ad728b Mon Sep 17 00:00:00 2001 From: walter Date: Fri, 22 Nov 2024 20:09:21 +0800 Subject: [PATCH 056/110] [feat](binlog) Support add/build/drop inverted index binlog (#44418) Related PR: https://github.com/selectdb/ccr-syncer/pull/252 --- .../apache/doris/alter/IndexChangeJob.java | 4 ++++ .../apache/doris/binlog/BinlogManager.java | 24 +++++++++++++++++++ .../org/apache/doris/persist/EditLog.java | 12 ++++++++-- .../TableAddOrDropInvertedIndicesInfo.java | 4 ++++ gensrc/thrift/FrontendService.thrift | 6 ++--- 5 files changed, 45 insertions(+), 5 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java index bb0c018dc36f364..a51cdb01699e239 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java @@ -469,6 +469,10 @@ private void replayCancelled(IndexChangeJob replayedJob) { LOG.info("cancel index job {}, err: {}", jobId, errMsg); } + public String toJson() { + return GsonUtils.GSON.toJson(this); + } + public static IndexChangeJob read(DataInput in) throws IOException { if (Env.getCurrentEnvJournalVersion() < FeMetaVersion.VERSION_122) { IndexChangeJob job = new IndexChangeJob(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java index 1f785713666437f..0fadfc2b542c85f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java @@ -18,6 +18,7 @@ package org.apache.doris.binlog; import org.apache.doris.alter.AlterJobV2; +import org.apache.doris.alter.IndexChangeJob; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; import org.apache.doris.common.Config; @@ -35,6 +36,7 @@ import org.apache.doris.persist.ReplacePartitionOperationLog; import org.apache.doris.persist.ReplaceTableOperationLog; import org.apache.doris.persist.TableAddOrDropColumnsInfo; +import org.apache.doris.persist.TableAddOrDropInvertedIndicesInfo; import org.apache.doris.persist.TableInfo; import org.apache.doris.persist.TableRenameColumnInfo; import org.apache.doris.persist.TruncateTableInfo; @@ -385,6 +387,28 @@ public void addReplaceTable(ReplaceTableOperationLog info, long commitSeq) { addBinlog(dbId, tableIds, commitSeq, timestamp, type, data, false, info); } + public void addModifyTableAddOrDropInvertedIndices(TableAddOrDropInvertedIndicesInfo info, long commitSeq) { + long dbId = info.getDbId(); + List tableIds = Lists.newArrayList(); + tableIds.add(info.getTableId()); + long timestamp = -1; + TBinlogType type = TBinlogType.MODIFY_TABLE_ADD_OR_DROP_INVERTED_INDICES; + String data = info.toJson(); + + addBinlog(dbId, tableIds, commitSeq, timestamp, type, data, false, info); + } + + public void addIndexChangeJob(IndexChangeJob indexChangeJob, long commitSeq) { + long dbId = indexChangeJob.getDbId(); + List tableIds = Lists.newArrayList(); + tableIds.add(indexChangeJob.getTableId()); + long timestamp = -1; + TBinlogType type = TBinlogType.INDEX_CHANGE_JOB; + String data = indexChangeJob.toJson(); + + addBinlog(dbId, tableIds, commitSeq, timestamp, type, data, false, indexChangeJob); + } + // get binlog by dbId, return first binlog.version > version public Pair getBinlog(long dbId, long tableId, long prevCommitSeq) { TStatus status = new TStatus(TStatusCode.OK); diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index 5ae6f62ebb20e93..96f5d43aed2cbd8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -986,11 +986,13 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { final TableAddOrDropInvertedIndicesInfo info = (TableAddOrDropInvertedIndicesInfo) journal.getData(); env.getSchemaChangeHandler().replayModifyTableAddOrDropInvertedIndices(info); + env.getBinlogManager().addModifyTableAddOrDropInvertedIndices(info, logId); break; } case OperationType.OP_INVERTED_INDEX_JOB: { IndexChangeJob indexChangeJob = (IndexChangeJob) journal.getData(); env.getSchemaChangeHandler().replayIndexChangeJob(indexChangeJob); + env.getBinlogManager().addIndexChangeJob(indexChangeJob, logId); break; } case OperationType.OP_CLEAN_LABEL: { @@ -2058,11 +2060,17 @@ public void logModifyTableAddOrDropColumns(TableAddOrDropColumnsInfo info) { } public void logModifyTableAddOrDropInvertedIndices(TableAddOrDropInvertedIndicesInfo info) { - logEdit(OperationType.OP_MODIFY_TABLE_ADD_OR_DROP_INVERTED_INDICES, info); + long logId = logEdit(OperationType.OP_MODIFY_TABLE_ADD_OR_DROP_INVERTED_INDICES, info); + LOG.info("walter log modify table add or drop inverted indices, infos: {}, json: {}", + info, info.toJson(), new RuntimeException("test")); + Env.getCurrentEnv().getBinlogManager().addModifyTableAddOrDropInvertedIndices(info, logId); } public void logIndexChangeJob(IndexChangeJob indexChangeJob) { - logEdit(OperationType.OP_INVERTED_INDEX_JOB, indexChangeJob); + long logId = logEdit(OperationType.OP_INVERTED_INDEX_JOB, indexChangeJob); + LOG.info("walter log inverted index job, infos: {}, json: {}", + indexChangeJob, indexChangeJob.toJson(), new RuntimeException("test")); + Env.getCurrentEnv().getBinlogManager().addIndexChangeJob(indexChangeJob, logId); } public void logCleanLabel(CleanLabelOperationLog log) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/TableAddOrDropInvertedIndicesInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/TableAddOrDropInvertedIndicesInfo.java index efdc3ab6e9eac28..39a90046d2419ef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/TableAddOrDropInvertedIndicesInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/TableAddOrDropInvertedIndicesInfo.java @@ -95,6 +95,10 @@ public long getJobId() { return jobId; } + public String toJson() { + return GsonUtils.GSON.toJson(this); + } + @Override public void write(DataOutput out) throws IOException { Text.writeString(out, GsonUtils.GSON.toJson(this)); diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 47b88552862254e..c79931fe8855aab 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -1192,6 +1192,8 @@ enum TBinlogType { MODIFY_COMMENT = 16, MODIFY_VIEW_DEF = 17, REPLACE_TABLE = 18, + MODIFY_TABLE_ADD_OR_DROP_INVERTED_INDICES = 19, + INDEX_CHANGE_JOB = 20, // Keep some IDs for allocation so that when new binlog types are added in the // future, the changes can be picked back to the old versions without breaking @@ -1208,9 +1210,7 @@ enum TBinlogType { // MODIFY_XXX = 17, // MIN_UNKNOWN = 18, // UNKNOWN_3 = 19, - MIN_UNKNOWN = 19, - UNKNOWN_4 = 20, - UNKNOWN_5 = 21, + MIN_UNKNOWN = 21, UNKNOWN_6 = 22, UNKNOWN_7 = 23, UNKNOWN_8 = 24, From 20d67100567dd926fdaa4ad203bf64d05bdbfcd8 Mon Sep 17 00:00:00 2001 From: abmdocrt Date: Fri, 22 Nov 2024 20:58:49 +0800 Subject: [PATCH 057/110] [Fix](checker) Fixed infinite loop after internal error in checker. (#44479) When the checker encounters an internal error, such as a transaction conflict, the return value will be less than 0 and the function will return immediately, but the related instance will not be removed from the map. Additionally, if the return value of the do check is not 0, the inverted check will not be performed. This PR fixes both of these issues. --- cloud/src/recycler/checker.cpp | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/cloud/src/recycler/checker.cpp b/cloud/src/recycler/checker.cpp index fe0887867b0052f..19a10d61c12d357 100644 --- a/cloud/src/recycler/checker.cpp +++ b/cloud/src/recycler/checker.cpp @@ -168,25 +168,17 @@ int Checker::start() { auto ctime_ms = duration_cast(system_clock::now().time_since_epoch()).count(); g_bvar_checker_enqueue_cost_s.put(instance_id, ctime_ms / 1000 - enqueue_time_s); - ret = checker->do_check(); + int ret1 = checker->do_check(); + int ret2 = 0; if (config::enable_inverted_check) { - if (ret == 0) { - ret = checker->do_inverted_check(); - } - } - - if (ret < 0) { - // If ret < 0, it means that a temporary error occurred during the check process. - // The check job should not be considered finished, and the next round of check job - // should be retried as soon as possible. - return; + ret2 = checker->do_inverted_check(); } // If instance checker has been aborted, don't finish this job if (!checker->stopped()) { finish_instance_recycle_job(txn_kv_.get(), check_job_key, instance.instance_id(), - ip_port_, ret == 0, ctime_ms); + ip_port_, ret1 == 0 && ret2 == 0, ctime_ms); } { std::lock_guard lock(mtx_); From 31d51d10693b39f5049549400bd9ac282e2f378b Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Fri, 22 Nov 2024 19:23:50 +0530 Subject: [PATCH 058/110] [Enhancement] (nereids)implement syncCommand in nereids (#44335) Issue Number: close #42814 --- .../org/apache/doris/nereids/DorisParser.g4 | 6 +- .../nereids/parser/LogicalPlanBuilder.java | 7 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../trees/plans/commands/SyncCommand.java | 58 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../test_nereids_other_commands.groovy | 20 +++++++ 6 files changed, 96 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/SyncCommand.java create mode 100644 regression-test/suites/nereids_p0/test_nereids_other_commands.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 7e727a7f0419617..47214c8c271af69 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -59,6 +59,7 @@ statementBase | supportedRefreshStatement #supportedRefreshStatementAlias | supportedShowStatement #supportedShowStatementAlias | supportedRecoverStatement #supportedRecoverStatementAlias + | supportedLoadStatement #supportedLoadfStatementAlias | unsupportedStatement #unsupported ; @@ -234,6 +235,10 @@ supportedShowStatement tabletIds+=INTEGER_VALUE (COMMA tabletIds+=INTEGER_VALUE)* #showTabletsBelong ; +supportedLoadStatement + : SYNC #sync + ; + unsupportedOtherStatement : HELP mark=identifierOrText #help | INSTALL PLUGIN FROM source=identifierOrText properties=propertyClause? #installPlugin @@ -371,7 +376,6 @@ unsupportedLoadStatement | SHOW ROUTINE LOAD TASK ((FROM | IN) database=identifier)? wildWhere? #showRoutineLoadTask | SHOW ALL? CREATE ROUTINE LOAD FOR label=multipartIdentifier #showCreateRoutineLoad | SHOW CREATE LOAD FOR label=multipartIdentifier #showCreateLoad - | SYNC #sync | importSequenceStatement #importSequenceStatementAlias | importPrecedingFilterStatement #importPrecedingFilterStatementAlias | importWhereStatement #importWhereStatementAlias diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index cd9279c90230486..5513b4f5ecbebda 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -245,6 +245,7 @@ import org.apache.doris.nereids.DorisParser.SubqueryContext; import org.apache.doris.nereids.DorisParser.SubqueryExpressionContext; import org.apache.doris.nereids.DorisParser.SupportedUnsetStatementContext; +import org.apache.doris.nereids.DorisParser.SyncContext; import org.apache.doris.nereids.DorisParser.SystemVariableContext; import org.apache.doris.nereids.DorisParser.TableAliasContext; import org.apache.doris.nereids.DorisParser.TableNameContext; @@ -499,6 +500,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowVariablesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowWhiteListCommand; +import org.apache.doris.nereids.trees.plans.commands.SyncCommand; import org.apache.doris.nereids.trees.plans.commands.UnsetDefaultStorageVaultCommand; import org.apache.doris.nereids.trees.plans.commands.UnsetVariableCommand; import org.apache.doris.nereids.trees.plans.commands.UnsupportedCommand; @@ -4368,6 +4370,11 @@ public LogicalPlan visitShowTableId(ShowTableIdContext ctx) { return new ShowTableIdCommand(tableId); } + @Override + public LogicalPlan visitSync(SyncContext ctx) { + return new SyncCommand(); + } + @Override public LogicalPlan visitShowDelete(ShowDeleteContext ctx) { String dbName = null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index b4a9aaa51f12218..4e62a7e7269be97 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -211,6 +211,7 @@ public enum PlanType { SHOW_VIEW_COMMAND, SHOW_WHITE_LIST_COMMAND, SHOW_TABLETS_BELONG_COMMAND, + SYNC_COMMAND, RECOVER_DATABASE_COMMAND, RECOVER_TABLE_COMMAND, RECOVER_PARTITION_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/SyncCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/SyncCommand.java new file mode 100644 index 000000000000000..9499899d39d9f8c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/SyncCommand.java @@ -0,0 +1,58 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.RedirectStatus; +import org.apache.doris.analysis.StmtType; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +/** + * sync command + */ +public class SyncCommand extends Command implements Redirect { + + /** + * constructor + */ + public SyncCommand() { + super(PlanType.SYNC_COMMAND); + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + // nothing to do + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitSyncCommand(this, context); + } + + @Override + public RedirectStatus toRedirectStatus() { + return RedirectStatus.FORWARD_WITH_SYNC; + } + + @Override + public StmtType stmtType() { + return StmtType.SYNC; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 0a7446075fa68cf..815c5c67030c343 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -91,6 +91,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowVariablesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowWhiteListCommand; +import org.apache.doris.nereids.trees.plans.commands.SyncCommand; import org.apache.doris.nereids.trees.plans.commands.UnsetDefaultStorageVaultCommand; import org.apache.doris.nereids.trees.plans.commands.UnsetVariableCommand; import org.apache.doris.nereids.trees.plans.commands.UnsupportedCommand; @@ -422,6 +423,10 @@ default R visitShowTrashCommand(ShowTrashCommand showTrashCommand, C context) { return visitCommand(showTrashCommand, context); } + default R visitSyncCommand(SyncCommand syncCommand, C context) { + return visitCommand(syncCommand, context); + } + default R visitShowDeleteCommand(ShowDeleteCommand showDeleteCommand, C context) { return visitCommand(showDeleteCommand, context); } diff --git a/regression-test/suites/nereids_p0/test_nereids_other_commands.groovy b/regression-test/suites/nereids_p0/test_nereids_other_commands.groovy new file mode 100644 index 000000000000000..3115166c2b28216 --- /dev/null +++ b/regression-test/suites/nereids_p0/test_nereids_other_commands.groovy @@ -0,0 +1,20 @@ +// 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_nereids_other_commands") { + checkNereidsExecute("sync;") +} From 362efda9fd3929062c7c2ea8aa2ab4c456bdb10d Mon Sep 17 00:00:00 2001 From: Siyang Tang Date: Sat, 23 Nov 2024 10:52:32 +0800 Subject: [PATCH 059/110] [chore](static-check) Add some cast checks for files under dir be/src/olap/ (#44398) ### What problem does this PR solve? Issue Number: related #44400 --- be/src/common/cast_set.h | 12 ++++++++ be/src/http/action/calc_file_crc_action.cpp | 4 +-- be/src/http/action/calc_file_crc_action.h | 2 +- be/src/olap/base_compaction.cpp | 6 +++- be/src/olap/base_tablet.cpp | 32 ++++++++++++++------- be/src/olap/base_tablet.h | 2 +- 6 files changed, 42 insertions(+), 16 deletions(-) diff --git a/be/src/common/cast_set.h b/be/src/common/cast_set.h index 08a9c8fbb7ebaae..dcc744d8122a3a4 100644 --- a/be/src/common/cast_set.h +++ b/be/src/common/cast_set.h @@ -61,6 +61,12 @@ void cast_set(T& a, U b) { a = static_cast(b); } +template + requires std::is_floating_point_v and std::is_integral_v +void cast_set(T& a, U b) { + a = static_cast(b); +} + template requires std::is_integral_v && std::is_integral_v T cast_set(U b) { @@ -70,4 +76,10 @@ T cast_set(U b) { return static_cast(b); } +template + requires std::is_floating_point_v and std::is_integral_v +T cast_set(U b) { + return static_cast(b); +} + } // namespace doris diff --git a/be/src/http/action/calc_file_crc_action.cpp b/be/src/http/action/calc_file_crc_action.cpp index 66ec96a2a9ac65a..123f55dd7fd7440 100644 --- a/be/src/http/action/calc_file_crc_action.cpp +++ b/be/src/http/action/calc_file_crc_action.cpp @@ -46,7 +46,7 @@ CalcFileCrcAction::CalcFileCrcAction(ExecEnv* exec_env, BaseStorageEngine& engin // calculate the crc value of the files in the tablet Status CalcFileCrcAction::_handle_calc_crc(HttpRequest* req, uint32_t* crc_value, int64_t* start_version, int64_t* end_version, - int32_t* rowset_count, int64_t* file_count) { + uint32_t* rowset_count, int64_t* file_count) { uint64_t tablet_id = 0; const auto& req_tablet_id = req->param(TABLET_ID_KEY); if (req_tablet_id.empty()) { @@ -110,7 +110,7 @@ void CalcFileCrcAction::handle(HttpRequest* req) { uint32_t crc_value = 0; int64_t start_version = 0; int64_t end_version = 0; - int32_t rowset_count = 0; + uint32_t rowset_count = 0; int64_t file_count = 0; MonotonicStopWatch timer; diff --git a/be/src/http/action/calc_file_crc_action.h b/be/src/http/action/calc_file_crc_action.h index 30df8bfe629cf3b..ea5fb894957d19a 100644 --- a/be/src/http/action/calc_file_crc_action.h +++ b/be/src/http/action/calc_file_crc_action.h @@ -44,7 +44,7 @@ class CalcFileCrcAction : public HttpHandlerWithAuth { private: Status _handle_calc_crc(HttpRequest* req, uint32_t* crc_value, int64_t* start_version, - int64_t* end_version, int32_t* rowset_count, int64_t* file_count); + int64_t* end_version, uint32_t* rowset_count, int64_t* file_count); private: BaseStorageEngine& _engine; diff --git a/be/src/olap/base_compaction.cpp b/be/src/olap/base_compaction.cpp index 8b9cbd75ed33b80..9e428f27b0b5769 100644 --- a/be/src/olap/base_compaction.cpp +++ b/be/src/olap/base_compaction.cpp @@ -23,6 +23,7 @@ #include #include +#include "common/cast_set.h" #include "common/config.h" #include "common/logging.h" #include "olap/compaction.h" @@ -35,6 +36,8 @@ #include "util/trace.h" namespace doris { +#include "common/compile_check_begin.h" + using namespace ErrorCode; BaseCompaction::BaseCompaction(StorageEngine& engine, const TabletSharedPtr& tablet) @@ -184,7 +187,8 @@ Status BaseCompaction::pick_rowsets_to_compact() { // set to 1 to void divide by zero base_size = 1; } - double cumulative_base_ratio = static_cast(cumulative_total_size) / base_size; + double cumulative_base_ratio = + cast_set(cumulative_total_size) / cast_set(base_size); if (cumulative_base_ratio > min_data_ratio) { VLOG_NOTICE << "satisfy the base compaction policy. tablet=" << _tablet->tablet_id() diff --git a/be/src/olap/base_tablet.cpp b/be/src/olap/base_tablet.cpp index 89886ec8b4bea79..2e70e4586cc7682 100644 --- a/be/src/olap/base_tablet.cpp +++ b/be/src/olap/base_tablet.cpp @@ -20,6 +20,10 @@ #include #include +#include +#include + +#include "common/cast_set.h" #include "common/logging.h" #include "common/status.h" #include "olap/calc_delete_bitmap_executor.h" @@ -45,6 +49,8 @@ #include "vec/jsonb/serialize.h" namespace doris { +#include "common/compile_check_begin.h" + using namespace ErrorCode; namespace { @@ -462,9 +468,9 @@ Status BaseTablet::lookup_row_key(const Slice& encoded_key, TabletSchema* latest RowLocation loc; for (size_t i = 0; i < specified_rowsets.size(); i++) { - auto& rs = specified_rowsets[i]; - auto& segments_key_bounds = rs->rowset_meta()->get_segments_key_bounds(); - int num_segments = rs->num_segments(); + const auto& rs = specified_rowsets[i]; + const auto& segments_key_bounds = rs->rowset_meta()->get_segments_key_bounds(); + int num_segments = cast_set(rs->num_segments()); DCHECK_EQ(segments_key_bounds.size(), num_segments); std::vector picked_segments; for (int i = num_segments - 1; i >= 0; i--) { @@ -671,7 +677,8 @@ Status BaseTablet::calc_segment_delete_bitmap(RowsetSharedPtr rowset, RowsetSharedPtr rowset_find; auto st = lookup_row_key(key, rowset_schema.get(), true, specified_rowsets, &loc, - dummy_version.first - 1, segment_caches, &rowset_find); + cast_set(dummy_version.first - 1), segment_caches, + &rowset_find); bool expected_st = st.ok() || st.is() || st.is(); // It's a defensive DCHECK, we need to exclude some common errors to avoid core-dump // while stress test @@ -1130,7 +1137,7 @@ Status BaseTablet::generate_new_block_for_flexible_partial_update( const signed char* delete_sign_column_data) { if (skipped) { if (delete_sign_column_data != nullptr && - delete_sign_column_data[read_index_old[idx]] != 0) { + delete_sign_column_data[read_index_old[cast_set(idx)]] != 0) { if (tablet_column.has_default_value()) { new_col->insert_from(default_value_col, 0); } else if (tablet_column.is_nullable()) { @@ -1300,7 +1307,8 @@ Status BaseTablet::check_delete_bitmap_correctness(DeleteBitmapPtr delete_bitmap for (const auto& rowset : *rowsets) { rapidjson::Value value; std::string version_str = rowset->get_rowset_info_str(); - value.SetString(version_str.c_str(), version_str.length(), + value.SetString(version_str.c_str(), + cast_set(version_str.length()), required_rowsets_arr.GetAllocator()); required_rowsets_arr.PushBack(value, required_rowsets_arr.GetAllocator()); } @@ -1313,7 +1321,8 @@ Status BaseTablet::check_delete_bitmap_correctness(DeleteBitmapPtr delete_bitmap for (const auto& rowset : rowsets) { rapidjson::Value value; std::string version_str = rowset->get_rowset_info_str(); - value.SetString(version_str.c_str(), version_str.length(), + value.SetString(version_str.c_str(), + cast_set(version_str.length()), required_rowsets_arr.GetAllocator()); required_rowsets_arr.PushBack(value, required_rowsets_arr.GetAllocator()); } @@ -1321,7 +1330,8 @@ Status BaseTablet::check_delete_bitmap_correctness(DeleteBitmapPtr delete_bitmap for (const auto& missing_rowset_id : missing_ids) { rapidjson::Value miss_value; std::string rowset_id_str = missing_rowset_id.to_string(); - miss_value.SetString(rowset_id_str.c_str(), rowset_id_str.length(), + miss_value.SetString(rowset_id_str.c_str(), + cast_set(rowset_id_str.length()), missing_rowsets_arr.GetAllocator()); missing_rowsets_arr.PushBack(miss_value, missing_rowsets_arr.GetAllocator()); } @@ -1725,7 +1735,7 @@ std::vector BaseTablet::get_snapshot_rowset(bool include_stale_ void BaseTablet::calc_consecutive_empty_rowsets( std::vector* empty_rowsets, const std::vector& candidate_rowsets, int limit) { - int len = candidate_rowsets.size(); + int len = cast_set(candidate_rowsets.size()); for (int i = 0; i < len - 1; ++i) { auto rowset = candidate_rowsets[i]; auto next_rowset = candidate_rowsets[i + 1]; @@ -1761,7 +1771,7 @@ void BaseTablet::calc_consecutive_empty_rowsets( } Status BaseTablet::calc_file_crc(uint32_t* crc_value, int64_t start_version, int64_t end_version, - int32_t* rowset_count, int64_t* file_count) { + uint32_t* rowset_count, int64_t* file_count) { Version v(start_version, end_version); std::vector rowsets; traverse_rowsets([&rowsets, &v](const auto& rs) { @@ -1771,7 +1781,7 @@ Status BaseTablet::calc_file_crc(uint32_t* crc_value, int64_t start_version, int } }); std::sort(rowsets.begin(), rowsets.end(), Rowset::comparator); - *rowset_count = rowsets.size(); + *rowset_count = cast_set(rowsets.size()); *crc_value = 0; *file_count = 0; diff --git a/be/src/olap/base_tablet.h b/be/src/olap/base_tablet.h index b5da0e3bf06be18..b6fc953e46082ed 100644 --- a/be/src/olap/base_tablet.h +++ b/be/src/olap/base_tablet.h @@ -289,7 +289,7 @@ class BaseTablet { } Status calc_file_crc(uint32_t* crc_value, int64_t start_version, int64_t end_version, - int32_t* rowset_count, int64_t* file_count); + uint32_t* rowset_count, int64_t* file_count); Status show_nested_index_file(std::string* json_meta); From 8ddf4c79a2b6673133b5e03ca31fabee9cb02e01 Mon Sep 17 00:00:00 2001 From: walter Date: Sun, 24 Nov 2024 11:34:10 +0800 Subject: [PATCH 060/110] [fix](index) Fix CREATE/DROP INDEX stmt toSql (#44494) Parsing `CREATE INDEX` and `DROP INDEX` will generate an `AlterTableStmt`, and the `AlterTableStmt ::toSql()` result is: `ALTER TABLE
CREATE INDEX ()` and `ALTER TABLE
DROP INDEX ON
` This PR corrects the output to `ALTER TABLE
ADD/DROP INDEX ...` --- .../apache/doris/analysis/AlterTableStmt.java | 4 +++ .../doris/analysis/CreateIndexClause.java | 4 +++ .../doris/analysis/DropIndexClause.java | 4 +++ .../doris/analysis/AlterTableStmtTest.java | 32 +++++++++++++++++++ 4 files changed, 44 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterTableStmt.java index aa60bcdf813df34..9290e0ee89417b0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterTableStmt.java @@ -212,6 +212,10 @@ public String toSql() { sb.append("DROP ROLLUP "); } sb.append(((AddRollupClause) op).getRollupName()); + } else if (op instanceof CreateIndexClause) { + sb.append(((CreateIndexClause) op).toSql(true)); + } else if (op instanceof DropIndexClause) { + sb.append(((DropIndexClause) op).toSql(true)); } else { sb.append(op.toSql()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateIndexClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateIndexClause.java index 16f90432e7b7ab3..62d051ca318d313 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateIndexClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateIndexClause.java @@ -88,6 +88,10 @@ public boolean needChangeMTMVState() { @Override public String toSql() { + return toSql(alter); + } + + public String toSql(boolean alter) { if (alter) { return "ADD " + indexDef.toSql(); } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropIndexClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropIndexClause.java index 783bed775be0f6d..25b4db9d16f00d1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropIndexClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropIndexClause.java @@ -80,6 +80,10 @@ public boolean needChangeMTMVState() { @Override public String toSql() { + return toSql(alter); + } + + public String toSql(boolean alter) { StringBuilder stringBuilder = new StringBuilder(); stringBuilder.append("DROP INDEX ").append("`" + indexName + "`"); if (!alter) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/AlterTableStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/AlterTableStmtTest.java index 9476df59cd14194..c968f21d6702004 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/AlterTableStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/AlterTableStmtTest.java @@ -126,6 +126,7 @@ public void testEnableFeature() throws UserException { @Test public void testCreateIndex() throws UserException { + // ALTER TABLE `db`.`table` ADD INDEX `index1` (`col1`) USING INVERTED COMMENT 'balabala' List ops = Lists.newArrayList(); ops.add(new CreateIndexClause( new TableName(InternalCatalog.INTERNAL_CATALOG_NAME, "db", "table"), @@ -137,8 +138,26 @@ public void testCreateIndex() throws UserException { stmt.toSql()); } + @Test + public void testCreateIndexStmt() throws UserException { + // CREATE INDEX `index1` ON `db`.`table` (`col1`) USING INVERTED COMMENT 'balabala' + CreateIndexClause createIndexClause = new CreateIndexClause( + new TableName(InternalCatalog.INTERNAL_CATALOG_NAME, "db", "table"), + new IndexDef("index1", false, Lists.newArrayList("col1"), IndexDef.IndexType.INVERTED, null, "balabala"), + false); + List ops = Lists.newArrayList(); + ops.add(createIndexClause); + AlterTableStmt stmt = new AlterTableStmt(new TableName(internalCtl, "testDb", "testTbl"), ops); + stmt.analyze(analyzer); + Assert.assertEquals("CREATE INDEX `index1` ON `db`.`table` (`col1`) USING INVERTED COMMENT 'balabala'", + createIndexClause.toSql()); + Assert.assertEquals("ALTER TABLE `testDb`.`testTbl` ADD INDEX `index1` (`col1`) USING INVERTED COMMENT 'balabala'", + stmt.toSql()); + } + @Test public void testDropIndex() throws UserException { + // ALTER TABLE `db`.`table` DROP INDEX `index1` List ops = Lists.newArrayList(); ops.add(new DropIndexClause("index1", false, new TableName(InternalCatalog.INTERNAL_CATALOG_NAME, "db", "table"), true)); @@ -146,4 +165,17 @@ public void testDropIndex() throws UserException { stmt.analyze(analyzer); Assert.assertEquals("ALTER TABLE `testDb`.`testTbl` DROP INDEX `index1`", stmt.toSql()); } + + @Test + public void testDropIndexStmt() throws UserException { + // DROP INDEX `index1` ON `db`.`table` + DropIndexClause dropIndexClause = new DropIndexClause("index1", false, + new TableName(InternalCatalog.INTERNAL_CATALOG_NAME, "db", "table"), false); + List ops = Lists.newArrayList(); + ops.add(dropIndexClause); + AlterTableStmt stmt = new AlterTableStmt(new TableName(internalCtl, "testDb", "testTbl"), ops); + stmt.analyze(analyzer); + Assert.assertEquals("DROP INDEX `index1` ON `db`.`table`", dropIndexClause.toSql()); + Assert.assertEquals("ALTER TABLE `testDb`.`testTbl` DROP INDEX `index1`", stmt.toSql()); + } } From ac6cd947c725296113cdd4e070fcff5cced65f27 Mon Sep 17 00:00:00 2001 From: deardeng Date: Sun, 24 Nov 2024 19:14:40 +0800 Subject: [PATCH 061/110] [fix](table) Disable create, alter auto bucket table with colocate (#44396) Colorate and auto bucket are naturally mutually exclusive, so diable it now --- .../java/org/apache/doris/catalog/Env.java | 3 + .../doris/datasource/InternalCatalog.java | 3 + .../table_p0/test_colocate_table.groovy | 63 +++++++++++++++++++ 3 files changed, 69 insertions(+) create mode 100644 regression-test/suites/table_p0/test_colocate_table.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index 85e041b88b52fb0..7597a7d256b4603 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -4868,6 +4868,9 @@ public void modifyTableColocate(Database db, OlapTable table, String assignedGro return; } } + if (!isReplay && table.isAutoBucket()) { + throw new DdlException("table " + table.getName() + " is auto buckets"); + } ColocateGroupSchema groupSchema = colocateTableIndex.getGroupSchema(fullAssignedGroupName); if (groupSchema == null) { // user set a new colocate group, diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index ac5cfd59a375883..3a9e96bade67f53 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -2938,6 +2938,9 @@ private boolean createOlapTable(Database db, CreateTableStmt stmt) throws UserEx if (defaultDistributionInfo.getType() == DistributionInfoType.RANDOM) { throw new AnalysisException("Random distribution for colocate table is unsupported"); } + if (isAutoBucket) { + throw new AnalysisException("Auto buckets for colocate table is unsupported"); + } String fullGroupName = GroupId.getFullGroupName(db.getId(), colocateGroup); ColocateGroupSchema groupSchema = Env.getCurrentColocateIndex().getGroupSchema(fullGroupName); if (groupSchema != null) { diff --git a/regression-test/suites/table_p0/test_colocate_table.groovy b/regression-test/suites/table_p0/test_colocate_table.groovy new file mode 100644 index 000000000000000..514b6fe2e8c6bf5 --- /dev/null +++ b/regression-test/suites/table_p0/test_colocate_table.groovy @@ -0,0 +1,63 @@ +// 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_colocate_table') { + def tbl = 'test_colocate_table' + sql "DROP TABLE IF EXISTS ${tbl} FORCE" + test { + sql """ + CREATE TABLE IF NOT EXISTS ${tbl} + ( + k1 date, + k2 int + ) + ENGINE=OLAP + UNIQUE KEY (k1,k2) + DISTRIBUTED BY HASH(k2) BUCKETS AUTO + PROPERTIES + ( + "replication_num" = "1", + "colocate_with" = "test_colocate_table_group" + ) + """ + + exception 'Auto buckets for colocate table is unsupported' + } + + sql """ + CREATE TABLE IF NOT EXISTS ${tbl} + ( + k1 date, + k2 int + ) + ENGINE=OLAP + UNIQUE KEY (k1,k2) + DISTRIBUTED BY HASH(k2) BUCKETS AUTO + PROPERTIES + ( + "replication_num" = "1" + ) + """ + + test { + sql "ALTER TABLE ${tbl} set ( 'colocate_with' = 'test_colocate_table_group') " + + exception 'is auto buckets' + } + + sql "DROP TABLE IF EXISTS ${tbl} FORCE" +} From 24e34d3d021c642e9ef2d6dddc937a89c021a5dd Mon Sep 17 00:00:00 2001 From: zhangyuan Date: Mon, 25 Nov 2024 09:56:38 +0800 Subject: [PATCH 062/110] [fix](sync mv) fix rewrite wrongly with sync mv (#39284) Aggregate table rewrite wrongly with sync mv: drop table if exists t1; CREATE TABLE IF NOT EXISTS t1 ( k int, a int, int_value int sum, char_value char(10) max, date_value date max ) ENGINE=OLAP aggregate KEY(k,a) DISTRIBUTED BY HASH(k) BUCKETS 2 properties("replication_num" = "1"); insert into t1 values (1,1,1,'a', '2020-12-01'), (2,2,2,'b', '2021-12-01'), (3,2,2,'c', '2022-12-01'), (4,2,4,'c', '2023-12-01'); mysql> select a from t1; +------+ | a | +------+ | 2 | | 1 | | 2 | | 2 | +------+ 4 rows in set (0.01 sec) CREATE MATERIALIZED VIEW v_t1 AS select k%2 as kk,a,max(date_value) from t1 group by kk, a; select JobName from mv_infos('database'='test') where Name = 'v_t1'; select sleep(2); // after rewire with v_t1, the result only have 3 rows mysql> select a from t1; +------+ | a | +------+ | 2 | | 2 | | 1 | +------+ 3 rows in set (0.01 sec) ## solution When rewriting query with aggregate table's sync mv, it is not only necessary to check the key size, but also to check all key columns; --- .../AbstractSelectMaterializedIndexRule.java | 12 ++ ...lectMaterializedIndexWithoutAggregate.java | 1 + .../mv/join/inner/inner_join_x.out | 25 ++++ .../mv/join/inner/inner_join_x.groovy | 128 ++++++++++++++++++ 4 files changed, 166 insertions(+) create mode 100644 regression-test/data/nereids_rules_p0/mv/join/inner/inner_join_x.out create mode 100644 regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join_x.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java index f17ab1c96bd9cfd..357883d1f7136af 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java @@ -144,6 +144,18 @@ protected static List getPrunedPredicates(List aggExpres return prunedExpr; } + protected static boolean containAllKeyColumns(OlapTable table, MaterializedIndex index) { + Set mvColNames = table.getKeyColumnsByIndexId(index.getId()).stream() + .map(c -> normalizeName(parseMvColumnToSql(c.getNameWithoutMvPrefix()))) + .collect(Collectors.toCollection(() -> new TreeSet(String.CASE_INSENSITIVE_ORDER))); + + Set keyColNames = table.getBaseSchemaKeyColumns().stream() + .map(c -> normalizeName(parseMvColumnToSql(c.getNameWithoutMvPrefix()))) + .collect(Collectors.toCollection(() -> new TreeSet(String.CASE_INSENSITIVE_ORDER))); + + return keyColNames.containsAll(mvColNames); + } + protected static boolean containAllRequiredColumns(MaterializedIndex index, LogicalOlapScan scan, Set requiredScanOutput, Set requiredExpr, Set predicateExpr) { OlapTable table = scan.getTable(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithoutAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithoutAggregate.java index 76278fe48ee9dd3..3a589108a09c1e2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithoutAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithoutAggregate.java @@ -234,6 +234,7 @@ public static LogicalOlapScan select( // So only base index and indexes that have all the keys could be used. List candidates = table.getVisibleIndex().stream() .filter(index -> table.getKeyColumnsByIndexId(index.getId()).size() == baseIndexKeySize) + .filter(index -> containAllKeyColumns(table, index)) .filter(index -> containAllRequiredColumns(index, scan, requiredScanOutputSupplier.get(), requiredExpr.get(), predicatesSupplier.get())) .collect(Collectors.toList()); diff --git a/regression-test/data/nereids_rules_p0/mv/join/inner/inner_join_x.out b/regression-test/data/nereids_rules_p0/mv/join/inner/inner_join_x.out new file mode 100644 index 000000000000000..201aca1c08f53f7 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/mv/join/inner/inner_join_x.out @@ -0,0 +1,25 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !query_before -- +1 +2 +2 +2 + +-- !query_after -- +1 +2 +2 +2 + +-- !query_before -- +1 +2 +2 +2 + +-- !query_after -- +1 +2 +2 +2 + diff --git a/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join_x.groovy b/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join_x.groovy new file mode 100644 index 000000000000000..5f8e150385a9529 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join_x.groovy @@ -0,0 +1,128 @@ +// 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("inner_join_x") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF" + sql "set enable_sync_mv_cost_based_rewrite=false" + + + // ======================= test table with aggregate key ============================ + sql """ + drop table if exists t1; + """ + + sql """ + CREATE TABLE IF NOT EXISTS t1 ( + k int, + a int, + int_value int sum, + char_value char(10) max, + date_value date max + ) + ENGINE=OLAP + aggregate KEY(k,a) + DISTRIBUTED BY HASH(k) BUCKETS 2 properties("replication_num" = "1") + + """ + + def mv_name="v_t1" + createMV ( """ + create materialized view ${mv_name} as select k%2 as kk,a, sum(int_value), max(date_value) from t1 group by kk, a; + """) + + sql """ + insert into t1 values + (1,1,1,'a', '2020-12-01'), + (2,2,2,'b', '2021-12-01'), + (3,2,2,'c', '2022-12-01'), + (4,2,4,'c', '2023-12-01'); + """ + + def query = """ + select a from t1 + """ + + explain { + sql("${query}") + notContains("${mv_name}(${mv_name})") + } + + order_qt_query_before "${query}" + + + sql """ DROP MATERIALIZED VIEW IF EXISTS ${mv_name} on t1""" + + order_qt_query_after "${query}" + + sql """ + drop table if exists t1 + """ + + // ======================= test table with duplicate key ============================ + sql """ + drop table if exists t1; + """ + + sql """ + CREATE TABLE IF NOT EXISTS t1 ( + k int, + a int, + int_value int, + char_value char(10), + date_value date + ) + ENGINE=OLAP + duplicate KEY(k,a) + DISTRIBUTED BY HASH(k) BUCKETS 2 properties("replication_num" = "1") + + """ + + mv_name="v_t1" + createMV ( """ + create materialized view ${mv_name} as select k%2 as kk,a, sum(int_value), max(date_value) from t1 group by kk, a; + """) + + sql """ + insert into t1 values + (1,1,1,'a', '2020-12-01'), + (2,2,2,'b', '2021-12-01'), + (3,2,2,'c', '2022-12-01'), + (4,2,4,'c', '2023-12-01'); + """ + + query = """ + select a from t1 + """ + + explain { + sql("${query}") + notContains("t1(${mv_name})") + } + + order_qt_query_before "${query}" + + + sql """ DROP MATERIALIZED VIEW IF EXISTS ${mv_name} on t1""" + + order_qt_query_after "${query}" + + sql """ + drop table if exists t1 + """ +} From 11379b5ab8f417747c90550c7c32c39e69ee4539 Mon Sep 17 00:00:00 2001 From: zclllhhjj Date: Mon, 25 Nov 2024 10:24:55 +0800 Subject: [PATCH 063/110] [testcase](auto-partition) Fix unintended partition creation when insert overwrite auto detect for auto partition table (#44451) add testcase of https://github.com/apache/doris/pull/44449 master code is right, so no change needed. --- .../test_iot_auto_detect_fail.groovy | 167 ++++++++++++++++++ 1 file changed, 167 insertions(+) create mode 100644 regression-test/suites/insert_overwrite_p0/test_iot_auto_detect_fail.groovy diff --git a/regression-test/suites/insert_overwrite_p0/test_iot_auto_detect_fail.groovy b/regression-test/suites/insert_overwrite_p0/test_iot_auto_detect_fail.groovy new file mode 100644 index 000000000000000..7bb2277f184726e --- /dev/null +++ b/regression-test/suites/insert_overwrite_p0/test_iot_auto_detect_fail.groovy @@ -0,0 +1,167 @@ +// 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_iot_auto_detect_fail") { + multi_sql """ + drop table if exists fail_src; + CREATE TABLE `fail_src` ( + `qsrq` int NULL, + `lsh` varchar(32) NULL, + `wth` bigint NULL, + `khh` varchar(16) NULL, + `dt` varchar(8) NULL +) ENGINE=OLAP +DUPLICATE KEY(`qsrq`, `lsh`) +AUTO PARTITION BY LIST (`dt`) +(PARTITION p202307078 VALUES IN ("20230707"), +PARTITION p202307108 VALUES IN ("20230710"), +PARTITION p202307118 VALUES IN ("20230711"), +PARTITION p202307128 VALUES IN ("20230712"), +PARTITION p202307138 VALUES IN ("20230713"), +PARTITION p202307148 VALUES IN ("20230714"), +PARTITION p202410088 VALUES IN ("20241008"), +PARTITION p202410098 VALUES IN ("20241009"), +PARTITION p202410108 VALUES IN ("20241010"), +PARTITION p202410118 VALUES IN ("20241011"), +PARTITION p202410148 VALUES IN ("20241014"), +PARTITION p202410158 VALUES IN ("20241015"), +PARTITION p202410168 VALUES IN ("20241016"), +PARTITION p202410178 VALUES IN ("20241017"), +PARTITION p202410188 VALUES IN ("20241018"), +PARTITION p202410218 VALUES IN ("20241021"), +PARTITION p202410228 VALUES IN ("20241022"), +PARTITION p202410238 VALUES IN ("20241023"), +PARTITION p202410248 VALUES IN ("20241024"), +PARTITION p202410258 VALUES IN ("20241025"), +PARTITION p202410288 VALUES IN ("20241028"), +PARTITION p202410298 VALUES IN ("20241029"), +PARTITION p202410308 VALUES IN ("20241030"), +PARTITION p202410318 VALUES IN ("20241031"), +PARTITION p202411018 VALUES IN ("20241101"), +PARTITION p202411028 VALUES IN ("20241102"), +PARTITION p202411038 VALUES IN ("20241103"), +PARTITION p202411048 VALUES IN ("20241104"), +PARTITION p202411058 VALUES IN ("20241105"), +PARTITION p202411068 VALUES IN ("20241106"), +PARTITION p202411078 VALUES IN ("20241107"), +PARTITION p202411088 VALUES IN ("20241108"), +PARTITION p202411118 VALUES IN ("20241111"), +PARTITION p202411128 VALUES IN ("20241112"), +PARTITION p202411138 VALUES IN ("20241113"), +PARTITION p202411148 VALUES IN ("20241114"), +PARTITION p202411158 VALUES IN ("20241115"), +PARTITION p202411248 VALUES IN ("20241124"), +PARTITION p202411258 VALUES IN ("20241125"), +PARTITION p202411268 VALUES IN ("20241126"), +PARTITION p202411278 VALUES IN ("20241127")) +DISTRIBUTED BY HASH(`khh`, `dt`) BUCKETS AUTO +PROPERTIES ( +"replication_allocation" = "tag.location.default: 1", +"min_load_replica_num" = "-1", +"is_being_synced" = "false", +"storage_medium" = "hdd", +"storage_format" = "V2", +"inverted_index_storage_format" = "V1", +"light_schema_change" = "true", +"disable_auto_compaction" = "false", +"enable_single_replica_compaction" = "false", +"group_commit_interval_ms" = "10000", +"group_commit_data_bytes" = "134217728" +); + +insert into fail_src values (3,'a',10,'b','20241128'),(4,'a',10,'b','20241128'),(5,'a',10,'b','20241128'),(6,'a',10,'b','20241128'),(7,'a',10,'b','20241128'),(8,'a',10,'b','20241128'),(9,'a',10,'b','20241128'); + +drop table if exists fail_tag; +CREATE TABLE `fail_tag` ( + `qsrq` int NULL, + `lsh` varchar(32) NULL, + `wth` bigint NULL, + `khh` varchar(16) NULL, + `dt` varchar(8) NULL +) ENGINE=OLAP +DUPLICATE KEY(`qsrq`, `lsh`) +AUTO PARTITION BY LIST (`dt`) +(PARTITION p202307078 VALUES IN ("20230707"), +PARTITION p202307108 VALUES IN ("20230710"), +PARTITION p202307118 VALUES IN ("20230711"), +PARTITION p202307128 VALUES IN ("20230712"), +PARTITION p202307138 VALUES IN ("20230713"), +PARTITION p202307148 VALUES IN ("20230714"), +PARTITION p202410088 VALUES IN ("20241008"), +PARTITION p202410098 VALUES IN ("20241009"), +PARTITION p202410108 VALUES IN ("20241010"), +PARTITION p202410118 VALUES IN ("20241011"), +PARTITION p202410148 VALUES IN ("20241014"), +PARTITION p202410158 VALUES IN ("20241015"), +PARTITION p202410168 VALUES IN ("20241016"), +PARTITION p202410178 VALUES IN ("20241017"), +PARTITION p202410188 VALUES IN ("20241018"), +PARTITION p202410218 VALUES IN ("20241021"), +PARTITION p202410228 VALUES IN ("20241022"), +PARTITION p202410238 VALUES IN ("20241023"), +PARTITION p202410248 VALUES IN ("20241024"), +PARTITION p202410258 VALUES IN ("20241025"), +PARTITION p202410288 VALUES IN ("20241028"), +PARTITION p202410298 VALUES IN ("20241029"), +PARTITION p202410308 VALUES IN ("20241030"), +PARTITION p202410318 VALUES IN ("20241031"), +PARTITION p202411018 VALUES IN ("20241101"), +PARTITION p202411028 VALUES IN ("20241102"), +PARTITION p202411038 VALUES IN ("20241103"), +PARTITION p202411048 VALUES IN ("20241104"), +PARTITION p202411058 VALUES IN ("20241105"), +PARTITION p202411068 VALUES IN ("20241106"), +PARTITION p202411078 VALUES IN ("20241107"), +PARTITION p202411088 VALUES IN ("20241108"), +PARTITION p202411118 VALUES IN ("20241111"), +PARTITION p202411128 VALUES IN ("20241112"), +PARTITION p202411138 VALUES IN ("20241113"), +PARTITION p202411148 VALUES IN ("20241114"), +PARTITION p202411158 VALUES IN ("20241115"), +PARTITION p202411248 VALUES IN ("20241124"), +PARTITION p202411258 VALUES IN ("20241125"), +PARTITION p202411268 VALUES IN ("20241126"), +PARTITION p202411278 VALUES IN ("20241127")) +DISTRIBUTED BY HASH(`khh`, `dt`) BUCKETS AUTO +PROPERTIES ( +"replication_allocation" = "tag.location.default: 1", +"min_load_replica_num" = "-1", +"is_being_synced" = "false", +"storage_medium" = "hdd", +"storage_format" = "V2", +"inverted_index_storage_format" = "V1", +"light_schema_change" = "true", +"disable_auto_compaction" = "false", +"enable_single_replica_compaction" = "false", +"group_commit_interval_ms" = "10000", +"group_commit_data_bytes" = "134217728" +); + """ + + test { + sql "insert overwrite table fail_tag PARTITION(*) select qsrq,lsh,wth,khh,dt from fail_src where dt='20241128';" + exception "Cannot found origin partitions" + } + test { + sql "insert overwrite table fail_tag PARTITION(*) select qsrq,lsh,wth,khh,dt from fail_src where dt='20241128';" + exception "Cannot found origin partitions" + } + test { + sql "insert overwrite table fail_tag PARTITION(*) select qsrq,lsh,wth,khh,dt from fail_src where dt='20241128';" + exception "Cannot found origin partitions" + } +} From 05b48d645bda0b928df4c1e9dd42d0fa0112bc03 Mon Sep 17 00:00:00 2001 From: Xinyi Zou Date: Mon, 25 Nov 2024 10:39:35 +0800 Subject: [PATCH 064/110] [fix](arrow-flight-sql) Arrow Flight support multiple endpoints (#44286) ### What problem does this PR solve? Problem Summary: By default, the query results of all BE nodes will be aggregated to one BE node. ADBC Client will only receive one endpoint and pull data from the BE node corresponding to this endpoint. `set global enable_parallel_result_sink=true;` to allow each BE to return query results separately. ADBC Client will receive multiple endpoints and pull data from each endpoint. --- .../org/apache/doris/qe/ConnectContext.java | 40 +---- .../java/org/apache/doris/qe/Coordinator.java | 32 ++-- .../apache/doris/qe/NereidsCoordinator.java | 29 ++-- .../arrowflight/DorisFlightSqlProducer.java | 77 +++++---- .../FlightSqlConnectProcessor.java | 154 +++++++++--------- .../results/FlightSqlEndpointsLocation.java | 65 ++++++++ 6 files changed, 228 insertions(+), 169 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/results/FlightSqlEndpointsLocation.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java index 5e0716da7d767fc..a16422ba9e5979f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java @@ -19,7 +19,6 @@ import org.apache.doris.analysis.BoolLiteral; import org.apache.doris.analysis.DecimalLiteral; -import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.FloatLiteral; import org.apache.doris.analysis.IntLiteral; import org.apache.doris.analysis.LiteralExpr; @@ -63,11 +62,11 @@ import org.apache.doris.plugin.AuditEvent.AuditEventBuilder; import org.apache.doris.resource.Tag; import org.apache.doris.service.arrowflight.results.FlightSqlChannel; +import org.apache.doris.service.arrowflight.results.FlightSqlEndpointsLocation; import org.apache.doris.statistics.ColumnStatistic; import org.apache.doris.statistics.Histogram; import org.apache.doris.system.Backend; import org.apache.doris.task.LoadTaskInfo; -import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TResultSinkType; import org.apache.doris.thrift.TStatusCode; import org.apache.doris.thrift.TUniqueId; @@ -134,10 +133,7 @@ public enum ConnectType { protected volatile String peerIdentity; private final Map preparedQuerys = new HashMap<>(); private String runningQuery; - private TNetworkAddress resultFlightServerAddr; - private TNetworkAddress resultInternalServiceAddr; - private ArrayList resultOutputExprs; - private TUniqueId finstId; + private final List flightSqlEndpointsLocations = Lists.newArrayList(); private boolean returnResultFromLocal = true; // mysql net protected volatile MysqlChannel mysqlChannel; @@ -730,36 +726,16 @@ public String getRunningQuery() { return runningQuery; } - public void setResultFlightServerAddr(TNetworkAddress resultFlightServerAddr) { - this.resultFlightServerAddr = resultFlightServerAddr; + public void addFlightSqlEndpointsLocation(FlightSqlEndpointsLocation flightSqlEndpointsLocation) { + this.flightSqlEndpointsLocations.add(flightSqlEndpointsLocation); } - public TNetworkAddress getResultFlightServerAddr() { - return resultFlightServerAddr; + public List getFlightSqlEndpointsLocations() { + return flightSqlEndpointsLocations; } - public void setResultInternalServiceAddr(TNetworkAddress resultInternalServiceAddr) { - this.resultInternalServiceAddr = resultInternalServiceAddr; - } - - public TNetworkAddress getResultInternalServiceAddr() { - return resultInternalServiceAddr; - } - - public void setResultOutputExprs(ArrayList resultOutputExprs) { - this.resultOutputExprs = resultOutputExprs; - } - - public ArrayList getResultOutputExprs() { - return resultOutputExprs; - } - - public void setFinstId(TUniqueId finstId) { - this.finstId = finstId; - } - - public TUniqueId getFinstId() { - return finstId; + public void clearFlightSqlEndpointsLocations() { + flightSqlEndpointsLocations.clear(); } public void setReturnResultFromLocal(boolean returnResultFromLocal) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index dee130886ec8f55..acd0fbe0daeaaf9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -83,6 +83,7 @@ import org.apache.doris.rpc.RpcException; import org.apache.doris.service.ExecuteEnv; import org.apache.doris.service.FrontendOptions; +import org.apache.doris.service.arrowflight.results.FlightSqlEndpointsLocation; import org.apache.doris.system.Backend; import org.apache.doris.system.SystemInfoService; import org.apache.doris.task.LoadEtlTask; @@ -733,29 +734,27 @@ protected void execInternal() throws Exception { enableParallelResultSink = queryOptions.isEnableParallelOutfile(); } - TNetworkAddress execBeAddr = topParams.instanceExecParams.get(0).host; Set addrs = new HashSet<>(); for (FInstanceExecParam param : topParams.instanceExecParams) { if (addrs.contains(param.host)) { continue; } addrs.add(param.host); - receivers.add(new ResultReceiver(queryId, param.instanceId, addressToBackendID.get(param.host), - toBrpcHost(param.host), this.timeoutDeadline, - context.getSessionVariable().getMaxMsgSizeOfResultReceiver(), enableParallelResultSink)); - } - - if (!context.isReturnResultFromLocal()) { - Preconditions.checkState(context.getConnectType().equals(ConnectType.ARROW_FLIGHT_SQL)); - if (enableParallelResultSink) { - context.setFinstId(queryId); + if (context.isReturnResultFromLocal()) { + receivers.add(new ResultReceiver(queryId, param.instanceId, addressToBackendID.get(param.host), + toBrpcHost(param.host), this.timeoutDeadline, + context.getSessionVariable().getMaxMsgSizeOfResultReceiver(), enableParallelResultSink)); } else { - context.setFinstId(topParams.instanceExecParams.get(0).instanceId); + Preconditions.checkState(context.getConnectType().equals(ConnectType.ARROW_FLIGHT_SQL)); + TUniqueId finstId; + if (enableParallelResultSink) { + finstId = queryId; + } else { + finstId = topParams.instanceExecParams.get(0).instanceId; + } + context.addFlightSqlEndpointsLocation(new FlightSqlEndpointsLocation(finstId, + toArrowFlightHost(param.host), toBrpcHost(param.host), fragments.get(0).getOutputExprs())); } - context.setFinstId(topParams.instanceExecParams.get(0).instanceId); - context.setResultFlightServerAddr(toArrowFlightHost(execBeAddr)); - context.setResultInternalServiceAddr(toBrpcHost(execBeAddr)); - context.setResultOutputExprs(fragments.get(0).getOutputExprs()); } LOG.info("dispatch result sink of query {} to {}", DebugUtil.printId(queryId), @@ -766,7 +765,8 @@ protected void execInternal() throws Exception { // set the broker address for OUTFILE sink ResultFileSink topResultFileSink = (ResultFileSink) topDataSink; FsBroker broker = Env.getCurrentEnv().getBrokerMgr() - .getBroker(topResultFileSink.getBrokerName(), execBeAddr.getHostname()); + .getBroker(topResultFileSink.getBrokerName(), + topParams.instanceExecParams.get(0).host.getHostname()); topResultFileSink.setBrokerAddr(broker.host, broker.port); } } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java index d718089fcabfeee..a9d6becc7fafe3f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java @@ -52,6 +52,7 @@ import org.apache.doris.qe.runtime.ThriftPlansBuilder; import org.apache.doris.resource.workloadgroup.QueryQueue; import org.apache.doris.resource.workloadgroup.QueueToken; +import org.apache.doris.service.arrowflight.results.FlightSqlEndpointsLocation; import org.apache.doris.system.Backend; import org.apache.doris.thrift.TErrorTabletInfo; import org.apache.doris.thrift.TNetworkAddress; @@ -90,7 +91,7 @@ public NereidsCoordinator(ConnectContext context, Analyzer analyzer, this.coordinatorContext.setJobProcessor(buildJobProcessor(coordinatorContext)); Preconditions.checkState(!planner.getFragments().isEmpty() - && coordinatorContext.instanceNum.get() > 0, "Fragment and Instance can not be empty˚"); + && coordinatorContext.instanceNum.get() > 0, "Fragment and Instance can not be empty˚"); } // broker load @@ -431,18 +432,22 @@ private void setForArrowFlight(CoordinatorContext coordinatorContext, PipelineDi if (dataSink instanceof ResultSink || dataSink instanceof ResultFileSink) { if (connectContext != null && !connectContext.isReturnResultFromLocal()) { Preconditions.checkState(connectContext.getConnectType().equals(ConnectType.ARROW_FLIGHT_SQL)); - - AssignedJob firstInstance = topPlan.getInstanceJobs().get(0); - BackendWorker worker = (BackendWorker) firstInstance.getAssignedWorker(); - Backend backend = worker.getBackend(); - - connectContext.setFinstId(firstInstance.instanceId()); - if (backend.getArrowFlightSqlPort() < 0) { - throw new IllegalStateException("be arrow_flight_sql_port cannot be empty."); + for (AssignedJob instance : topPlan.getInstanceJobs()) { + BackendWorker worker = (BackendWorker) instance.getAssignedWorker(); + Backend backend = worker.getBackend(); + if (backend.getArrowFlightSqlPort() < 0) { + throw new IllegalStateException("be arrow_flight_sql_port cannot be empty."); + } + TUniqueId finstId; + if (connectContext.getSessionVariable().enableParallelResultSink()) { + finstId = getQueryId(); + } else { + finstId = instance.instanceId(); + } + connectContext.addFlightSqlEndpointsLocation(new FlightSqlEndpointsLocation(finstId, + backend.getArrowFlightAddress(), backend.getBrpcAddress(), + topPlan.getFragmentJob().getFragment().getOutputExprs())); } - connectContext.setResultFlightServerAddr(backend.getArrowFlightAddress()); - connectContext.setResultInternalServiceAddr(backend.getBrpcAddress()); - connectContext.setResultOutputExprs(topPlan.getFragmentJob().getFragment().getOutputExprs()); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/DorisFlightSqlProducer.java b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/DorisFlightSqlProducer.java index 9d44a55b0816457..b968ab04c57c83b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/DorisFlightSqlProducer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/DorisFlightSqlProducer.java @@ -25,11 +25,13 @@ import org.apache.doris.mysql.MysqlCommand; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.QueryState.MysqlStateType; +import org.apache.doris.service.arrowflight.results.FlightSqlEndpointsLocation; import org.apache.doris.service.arrowflight.results.FlightSqlResultCacheEntry; import org.apache.doris.service.arrowflight.sessions.FlightSessionsManager; import org.apache.doris.thrift.TUniqueId; import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; import com.google.protobuf.Any; import com.google.protobuf.ByteString; import com.google.protobuf.Message; @@ -187,6 +189,7 @@ private FlightInfo executeQueryStatement(String peerIdentity, ConnectContext con Preconditions.checkState(!query.isEmpty()); // After the previous query was executed, there was no getStreamStatement to take away the result. connectContext.getFlightSqlChannel().reset(); + connectContext.clearFlightSqlEndpointsLocations(); try (FlightSqlConnectProcessor flightSQLConnectProcessor = new FlightSqlConnectProcessor(connectContext)) { flightSQLConnectProcessor.handleQuery(query); if (connectContext.getState().getStateType() == MysqlStateType.ERR) { @@ -225,50 +228,52 @@ private FlightInfo executeQueryStatement(String peerIdentity, ConnectContext con } } else { // Now only query stmt will pull results from BE. - Schema schema = flightSQLConnectProcessor.fetchArrowFlightSchema(5000); - if (schema == null) { + flightSQLConnectProcessor.fetchArrowFlightSchema(5000); + if (flightSQLConnectProcessor.getArrowSchema() == null) { throw CallStatus.INTERNAL.withDescription("fetch arrow flight schema is null") .toRuntimeException(); } - TUniqueId queryId = connectContext.queryId(); - if (!connectContext.getSessionVariable().enableParallelResultSink()) { - // only one instance - queryId = connectContext.getFinstId(); - } - // Ticket contains the IP and Brpc Port of the Doris BE node where the query result is located. - final ByteString handle = ByteString.copyFromUtf8( - DebugUtil.printId(queryId) + "&" + connectContext.getResultInternalServiceAddr().hostname - + "&" + connectContext.getResultInternalServiceAddr().port + "&" + query); - TicketStatementQuery ticketStatement = TicketStatementQuery.newBuilder().setStatementHandle(handle) - .build(); - Ticket ticket = new Ticket(Any.pack(ticketStatement).toByteArray()); - // TODO Support multiple endpoints. - Location location; - if (flightSQLConnectProcessor.getPublicAccessAddr().isSetHostname()) { - // In a production environment, it is often inconvenient to expose Doris BE nodes - // to the external network. - // However, a reverse proxy (such as nginx) can be added to all Doris BE nodes, - // and the external client will be randomly routed to a Doris BE node when connecting to nginx. - // The query results of Arrow Flight SQL will be randomly saved on a Doris BE node. - // If it is different from the Doris BE node randomly routed by nginx, - // data forwarding needs to be done inside the Doris BE node. - if (flightSQLConnectProcessor.getPublicAccessAddr().isSetPort()) { - location = Location.forGrpcInsecure( - flightSQLConnectProcessor.getPublicAccessAddr().hostname, - flightSQLConnectProcessor.getPublicAccessAddr().port); + List endpoints = Lists.newArrayList(); + for (FlightSqlEndpointsLocation endpointLoc : connectContext.getFlightSqlEndpointsLocations()) { + TUniqueId tid = endpointLoc.getFinstId(); + // Ticket contains the IP and Brpc Port of the Doris BE node where the query result is located. + final ByteString handle = ByteString.copyFromUtf8( + DebugUtil.printId(tid) + "&" + endpointLoc.getResultInternalServiceAddr().hostname + "&" + + endpointLoc.getResultInternalServiceAddr().port + "&" + query); + TicketStatementQuery ticketStatement = TicketStatementQuery.newBuilder() + .setStatementHandle(handle).build(); + Ticket ticket = new Ticket(Any.pack(ticketStatement).toByteArray()); + Location location; + if (endpointLoc.getResultPublicAccessAddr().isSetHostname()) { + // In a production environment, it is often inconvenient to expose Doris BE nodes + // to the external network. + // However, a reverse proxy (such as nginx) can be added to all Doris BE nodes, + // and the external client will be randomly routed to a Doris BE node when connecting + // to nginx. + // The query results of Arrow Flight SQL will be randomly saved on a Doris BE node. + // If it is different from the Doris BE node randomly routed by nginx, + // data forwarding needs to be done inside the Doris BE node. + if (endpointLoc.getResultPublicAccessAddr().isSetPort()) { + location = Location.forGrpcInsecure(endpointLoc.getResultPublicAccessAddr().hostname, + endpointLoc.getResultPublicAccessAddr().port); + } else { + location = Location.forGrpcInsecure(endpointLoc.getResultPublicAccessAddr().hostname, + endpointLoc.getResultFlightServerAddr().port); + } } else { - location = Location.forGrpcInsecure( - flightSQLConnectProcessor.getPublicAccessAddr().hostname, - connectContext.getResultFlightServerAddr().port); + location = Location.forGrpcInsecure(endpointLoc.getResultFlightServerAddr().hostname, + endpointLoc.getResultFlightServerAddr().port); } - } else { - location = Location.forGrpcInsecure(connectContext.getResultFlightServerAddr().hostname, - connectContext.getResultFlightServerAddr().port); + // By default, the query results of all BE nodes will be aggregated to one BE node. + // ADBC Client will only receive one endpoint and pull data from the BE node + // corresponding to this endpoint. + // `set global enable_parallel_result_sink=true;` to allow each BE to return query results + // separately. ADBC Client will receive multiple endpoints and pull data from each endpoint. + endpoints.add(new FlightEndpoint(ticket, location)); } - List endpoints = Collections.singletonList(new FlightEndpoint(ticket, location)); // TODO Set in BE callback after query end, Client will not callback. - return new FlightInfo(schema, descriptor, endpoints, -1, -1); + return new FlightInfo(flightSQLConnectProcessor.getArrowSchema(), descriptor, endpoints, -1, -1); } } } catch (Exception e) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightSqlConnectProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightSqlConnectProcessor.java index db5213cb7d4d08f..3fba602a1c1e2f5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightSqlConnectProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightSqlConnectProcessor.java @@ -31,6 +31,7 @@ import org.apache.doris.qe.StmtExecutor; import org.apache.doris.rpc.BackendServiceProxy; import org.apache.doris.rpc.RpcException; +import org.apache.doris.service.arrowflight.results.FlightSqlEndpointsLocation; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TStatusCode; import org.apache.doris.thrift.TUniqueId; @@ -58,7 +59,7 @@ */ public class FlightSqlConnectProcessor extends ConnectProcessor implements AutoCloseable { private static final Logger LOG = LogManager.getLogger(FlightSqlConnectProcessor.class); - private TNetworkAddress publicAccessAddr = new TNetworkAddress(); + private Schema arrowSchema; public FlightSqlConnectProcessor(ConnectContext context) { super(context); @@ -67,8 +68,8 @@ public FlightSqlConnectProcessor(ConnectContext context) { context.setReturnResultFromLocal(true); } - public TNetworkAddress getPublicAccessAddr() { - return publicAccessAddr; + public Schema getArrowSchema() { + return arrowSchema; } public void prepare(MysqlCommand command) { @@ -107,80 +108,87 @@ public void handleQuery(String query) throws ConnectionException { // handleFieldList(tableName); // } - public Schema fetchArrowFlightSchema(int timeoutMs) { - TNetworkAddress address = ctx.getResultInternalServiceAddr(); - TUniqueId tid; - if (ctx.getSessionVariable().enableParallelResultSink()) { - tid = ctx.queryId(); - } else { - // only one instance - tid = ctx.getFinstId(); + public void fetchArrowFlightSchema(int timeoutMs) { + if (ctx.getFlightSqlEndpointsLocations().isEmpty()) { + throw new RuntimeException("fetch arrow flight schema failed, no FlightSqlEndpointsLocations."); } - ArrayList resultOutputExprs = ctx.getResultOutputExprs(); - Types.PUniqueId queryId = Types.PUniqueId.newBuilder().setHi(tid.hi).setLo(tid.lo).build(); - try { - InternalService.PFetchArrowFlightSchemaRequest request = - InternalService.PFetchArrowFlightSchemaRequest.newBuilder() - .setFinstId(queryId) - .build(); - - Future future - = BackendServiceProxy.getInstance().fetchArrowFlightSchema(address, request); - InternalService.PFetchArrowFlightSchemaResult pResult; - pResult = future.get(timeoutMs, TimeUnit.MILLISECONDS); - if (pResult == null) { - throw new RuntimeException(String.format("fetch arrow flight schema timeout, queryId: %s", - DebugUtil.printId(tid))); - } - Status resultStatus = new Status(pResult.getStatus()); - if (resultStatus.getErrorCode() != TStatusCode.OK) { - throw new RuntimeException(String.format("fetch arrow flight schema failed, queryId: %s, errmsg: %s", - DebugUtil.printId(tid), resultStatus)); - } - if (pResult.hasBeArrowFlightIp()) { - publicAccessAddr.setHostname(pResult.getBeArrowFlightIp().toStringUtf8()); - } - if (pResult.hasBeArrowFlightPort()) { - publicAccessAddr.setPort(pResult.getBeArrowFlightPort()); - } - if (pResult.hasSchema() && pResult.getSchema().size() > 0) { - RootAllocator rootAllocator = new RootAllocator(Integer.MAX_VALUE); - ArrowStreamReader arrowStreamReader = new ArrowStreamReader( - new ByteArrayInputStream(pResult.getSchema().toByteArray()), - rootAllocator - ); - try { - VectorSchemaRoot root = arrowStreamReader.getVectorSchemaRoot(); - List fieldVectors = root.getFieldVectors(); - if (fieldVectors.size() != resultOutputExprs.size()) { - throw new RuntimeException(String.format( - "Schema size %s' is not equal to arrow field size %s, queryId: %s.", - fieldVectors.size(), resultOutputExprs.size(), DebugUtil.printId(tid))); + for (FlightSqlEndpointsLocation endpointLoc : ctx.getFlightSqlEndpointsLocations()) { + TNetworkAddress address = endpointLoc.getResultInternalServiceAddr(); + TUniqueId tid = endpointLoc.getFinstId(); + ArrayList resultOutputExprs = endpointLoc.getResultOutputExprs(); + Types.PUniqueId queryId = Types.PUniqueId.newBuilder().setHi(tid.hi).setLo(tid.lo).build(); + try { + InternalService.PFetchArrowFlightSchemaRequest request + = InternalService.PFetchArrowFlightSchemaRequest.newBuilder().setFinstId(queryId).build(); + + Future future = BackendServiceProxy.getInstance() + .fetchArrowFlightSchema(address, request); + InternalService.PFetchArrowFlightSchemaResult pResult; + pResult = future.get(timeoutMs, TimeUnit.MILLISECONDS); + if (pResult == null) { + throw new RuntimeException( + String.format("fetch arrow flight schema timeout, queryId: %s", DebugUtil.printId(tid))); + } + Status resultStatus = new Status(pResult.getStatus()); + if (resultStatus.getErrorCode() != TStatusCode.OK) { + throw new RuntimeException( + String.format("fetch arrow flight schema failed, queryId: %s, errmsg: %s", + DebugUtil.printId(tid), resultStatus)); + } + + TNetworkAddress resultPublicAccessAddr = new TNetworkAddress(); + if (pResult.hasBeArrowFlightIp()) { + resultPublicAccessAddr.setHostname(pResult.getBeArrowFlightIp().toStringUtf8()); + } + if (pResult.hasBeArrowFlightPort()) { + resultPublicAccessAddr.setPort(pResult.getBeArrowFlightPort()); + } + endpointLoc.setResultPublicAccessAddr(resultPublicAccessAddr); + if (pResult.hasSchema() && pResult.getSchema().size() > 0) { + RootAllocator rootAllocator = new RootAllocator(Integer.MAX_VALUE); + ArrowStreamReader arrowStreamReader = new ArrowStreamReader( + new ByteArrayInputStream(pResult.getSchema().toByteArray()), rootAllocator); + try { + Schema schema; + VectorSchemaRoot root = arrowStreamReader.getVectorSchemaRoot(); + List fieldVectors = root.getFieldVectors(); + if (fieldVectors.size() != resultOutputExprs.size()) { + throw new RuntimeException( + String.format("Schema size %s' is not equal to arrow field size %s, queryId: %s.", + fieldVectors.size(), resultOutputExprs.size(), DebugUtil.printId(tid))); + } + schema = root.getSchema(); + if (arrowSchema == null) { + arrowSchema = schema; + } else if (!arrowSchema.equals(schema)) { + throw new RuntimeException(String.format( + "The schema returned by results BE is different, first schema: %s, " + + "new schema: %s, queryId: %s,backend: %s", arrowSchema, schema, + DebugUtil.printId(tid), address)); + } + } catch (Exception e) { + throw new RuntimeException("Read Arrow Flight Schema failed.", e); } - return root.getSchema(); - } catch (Exception e) { - throw new RuntimeException("Read Arrow Flight Schema failed.", e); + } else { + throw new RuntimeException( + String.format("get empty arrow flight schema, queryId: %s", DebugUtil.printId(tid))); } - } else { - throw new RuntimeException(String.format("get empty arrow flight schema, queryId: %s", - DebugUtil.printId(tid))); + } catch (RpcException e) { + throw new RuntimeException( + String.format("arrow flight schema fetch catch rpc exception, queryId: %s,backend: %s", + DebugUtil.printId(tid), address), e); + } catch (InterruptedException e) { + throw new RuntimeException( + String.format("arrow flight schema future get interrupted exception, queryId: %s,backend: %s", + DebugUtil.printId(tid), address), e); + } catch (ExecutionException e) { + throw new RuntimeException( + String.format("arrow flight schema future get execution exception, queryId: %s,backend: %s", + DebugUtil.printId(tid), address), e); + } catch (TimeoutException e) { + throw new RuntimeException(String.format("arrow flight schema fetch timeout, queryId: %s,backend: %s", + DebugUtil.printId(tid), address), e); } - } catch (RpcException e) { - throw new RuntimeException(String.format( - "arrow flight schema fetch catch rpc exception, queryId: %s,backend: %s", - DebugUtil.printId(tid), address), e); - } catch (InterruptedException e) { - throw new RuntimeException(String.format( - "arrow flight schema future get interrupted exception, queryId: %s,backend: %s", - DebugUtil.printId(tid), address), e); - } catch (ExecutionException e) { - throw new RuntimeException(String.format( - "arrow flight schema future get execution exception, queryId: %s,backend: %s", - DebugUtil.printId(tid), address), e); - } catch (TimeoutException e) { - throw new RuntimeException(String.format( - "arrow flight schema fetch timeout, queryId: %s,backend: %s", - DebugUtil.printId(tid), address), e); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/results/FlightSqlEndpointsLocation.java b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/results/FlightSqlEndpointsLocation.java new file mode 100644 index 000000000000000..61adc797cc5dc4f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/results/FlightSqlEndpointsLocation.java @@ -0,0 +1,65 @@ +// 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.service.arrowflight.results; + +import org.apache.doris.analysis.Expr; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TUniqueId; + +import java.util.ArrayList; + +public class FlightSqlEndpointsLocation { + private TUniqueId finstId; + private TNetworkAddress resultFlightServerAddr; + private TNetworkAddress resultInternalServiceAddr; + private TNetworkAddress resultPublicAccessAddr; + private ArrayList resultOutputExprs; + + public FlightSqlEndpointsLocation(TUniqueId finstId, TNetworkAddress resultFlightServerAddr, + TNetworkAddress resultInternalServiceAddr, ArrayList resultOutputExprs) { + this.finstId = finstId; + this.resultFlightServerAddr = resultFlightServerAddr; + this.resultInternalServiceAddr = resultInternalServiceAddr; + this.resultPublicAccessAddr = new TNetworkAddress(); + this.resultOutputExprs = resultOutputExprs; + } + + public TUniqueId getFinstId() { + return finstId; + } + + public TNetworkAddress getResultFlightServerAddr() { + return resultFlightServerAddr; + } + + public TNetworkAddress getResultInternalServiceAddr() { + return resultInternalServiceAddr; + } + + public void setResultPublicAccessAddr(TNetworkAddress resultPublicAccessAddr) { + this.resultPublicAccessAddr = resultPublicAccessAddr; + } + + public TNetworkAddress getResultPublicAccessAddr() { + return resultPublicAccessAddr; + } + + public ArrayList getResultOutputExprs() { + return resultOutputExprs; + } +} From 659d6eb4422744ffe4f2a0215e95d060919dbda9 Mon Sep 17 00:00:00 2001 From: Pxl Date: Mon, 25 Nov 2024 11:15:46 +0800 Subject: [PATCH 065/110] [Improvement](column) optimization for insert_from_multi_column (#44483) ### What problem does this PR solve? 1. reduce virtual function overhead 2. use reserve at start before: - MergeGetNext: 504.852ms after: - MergeGetNext: 417.681ms --- be/src/vec/columns/column.cpp | 7 ------- be/src/vec/columns/column.h | 7 +++++-- be/src/vec/columns/column_impl.h | 9 +++++++++ be/src/vec/common/cow.h | 13 +++++++++++-- 4 files changed, 25 insertions(+), 11 deletions(-) diff --git a/be/src/vec/columns/column.cpp b/be/src/vec/columns/column.cpp index ec135fe65af6f50..57fd127f52de420 100644 --- a/be/src/vec/columns/column.cpp +++ b/be/src/vec/columns/column.cpp @@ -46,13 +46,6 @@ void IColumn::insert_from(const IColumn& src, size_t n) { insert(src[n]); } -void IColumn::insert_from_multi_column(const std::vector& srcs, - std::vector positions) { - for (size_t i = 0; i < srcs.size(); ++i) { - insert_from(*srcs[i], positions[i]); - } -} - void IColumn::sort_column(const ColumnSorter* sorter, EqualFlags& flags, IColumn::Permutation& perms, EqualRange& range, bool last_column) const { sorter->sort_column(static_cast(*this), flags, perms, range, last_column); diff --git a/be/src/vec/columns/column.h b/be/src/vec/columns/column.h index e92c246741f2e4d..19462b182bd2fd5 100644 --- a/be/src/vec/columns/column.h +++ b/be/src/vec/columns/column.h @@ -221,8 +221,8 @@ class IColumn : public COW { // insert the data of target columns into self column according to positions // positions[i] means index of srcs whitch need to insert_from // the virtual function overhead of multiple calls to insert_from can be reduced to once - void insert_from_multi_column(const std::vector& srcs, - std::vector positions); + virtual void insert_from_multi_column(const std::vector& srcs, + const std::vector& positions) = 0; /// Appends a batch elements from other column with the same type /// indices_begin + indices_end represent the row indices of column src @@ -706,6 +706,9 @@ class IColumn : public COW { template void append_data_by_selector_impl(MutablePtr& res, const Selector& selector, size_t begin, size_t end) const; + template + void insert_from_multi_column_impl(const std::vector& srcs, + const std::vector& positions); }; using ColumnPtr = IColumn::Ptr; diff --git a/be/src/vec/columns/column_impl.h b/be/src/vec/columns/column_impl.h index 470825851e1e9d1..f7abbf2a023ba47 100644 --- a/be/src/vec/columns/column_impl.h +++ b/be/src/vec/columns/column_impl.h @@ -57,4 +57,13 @@ void IColumn::append_data_by_selector_impl(MutablePtr& res, const Selector& sele append_data_by_selector_impl(res, selector, 0, selector.size()); } +template +void IColumn::insert_from_multi_column_impl(const std::vector& srcs, + const std::vector& positions) { + reserve(size() + srcs.size()); + for (size_t i = 0; i < srcs.size(); ++i) { + static_cast(*this).insert_from(*srcs[i], positions[i]); + } +} + } // namespace doris::vectorized diff --git a/be/src/vec/common/cow.h b/be/src/vec/common/cow.h index 0756ef3fce0371a..95df7694f227d9e 100644 --- a/be/src/vec/common/cow.h +++ b/be/src/vec/common/cow.h @@ -23,6 +23,9 @@ #include #include #include +#include + +namespace doris { /** Copy-on-write shared ptr. * Allows to work with shared immutable objects and sometimes unshare and mutate you own unique copy. @@ -392,7 +395,7 @@ class COW { * * See example in "cow_columns.cpp". */ -namespace doris::vectorized { +namespace vectorized { class IColumn; } template @@ -422,8 +425,14 @@ class COWHelper : public Base { this->template append_data_by_selector_impl(res, selector, begin, end); } + void insert_from_multi_column(const std::vector& srcs, + const std::vector& positions) override { + this->template insert_from_multi_column_impl(srcs, positions); + } + protected: MutablePtr shallow_mutate() const { return MutablePtr(static_cast(Base::shallow_mutate().get())); } -}; \ No newline at end of file +}; +} // namespace doris \ No newline at end of file From 7f436ae80f8d8503ce6b63ef164ed3ce85a685fd Mon Sep 17 00:00:00 2001 From: zhangstar333 Date: Mon, 25 Nov 2024 11:24:04 +0800 Subject: [PATCH 066/110] [Bug](function)window funnel function should support datetimev2 type get scale info (#44420) ### What problem does this PR solve? Problem Summary: before the window funnel function only support datetime type, which can't get scale info of really it's should support datetimev2 type --- .../functions/agg/WindowFunnel.java | 4 ++++ .../nereids_p0/aggregate/window_funnel.out | 3 +++ .../nereids_p0/aggregate/window_funnel.groovy | 21 +++++++++++++++++++ 3 files changed, 28 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/WindowFunnel.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/WindowFunnel.java index 11d920530e8d08e..7679b5a49052df3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/WindowFunnel.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/WindowFunnel.java @@ -44,8 +44,12 @@ public class WindowFunnel extends NullableAggregateFunction implements ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(IntegerType.INSTANCE) + .varArgs(BigIntType.INSTANCE, StringType.INSTANCE, DateTimeV2Type.SYSTEM_DEFAULT, + BooleanType.INSTANCE), FunctionSignature.ret(IntegerType.INSTANCE) .varArgs(BigIntType.INSTANCE, StringType.INSTANCE, DateTimeType.INSTANCE, BooleanType.INSTANCE) + ); /** diff --git a/regression-test/data/nereids_p0/aggregate/window_funnel.out b/regression-test/data/nereids_p0/aggregate/window_funnel.out index 0bbe397fe3ecb44..f9c24637999e773 100644 --- a/regression-test/data/nereids_p0/aggregate/window_funnel.out +++ b/regression-test/data/nereids_p0/aggregate/window_funnel.out @@ -128,3 +128,6 @@ 100126 2 100127 2 +-- !window_funnel_datetimev2 -- +2 + diff --git a/regression-test/suites/nereids_p0/aggregate/window_funnel.groovy b/regression-test/suites/nereids_p0/aggregate/window_funnel.groovy index 378311a575fd74b..186e65461da9cce 100644 --- a/regression-test/suites/nereids_p0/aggregate/window_funnel.groovy +++ b/regression-test/suites/nereids_p0/aggregate/window_funnel.groovy @@ -720,4 +720,25 @@ suite("window_funnel") { order BY user_id """ + sql """ DROP TABLE IF EXISTS windowfunnel_test_1 """ + sql """ + CREATE TABLE windowfunnel_test_1 ( + `xwho` varchar(50) NULL COMMENT 'xwho', + `xwhen` datetime(3) COMMENT 'xwhen', + `xwhat` int NULL COMMENT 'xwhat' + ) DUPLICATE KEY(xwho) + DISTRIBUTED BY HASH(xwho) BUCKETS 3 + PROPERTIES ("replication_num" = "1"); + """ + sql """ INSERT INTO windowfunnel_test_1 (xwho, xwhen, xwhat) VALUES + ('1', '2022-03-12 10:41:00.111', 1), + ('1', '2022-03-12 10:41:00.888', 2), + ('1', '2022-03-12 13:28:02.177', 3), + ('1', '2022-03-12 16:15:01.444', 4); """ + explain { + sql("""select window_funnel(3600 * 3, 'deduplication', t.xwhen, t.xwhat = 1, t.xwhat = 2 ) AS level from windowfunnel_test_1 t;""") + notContains("cast") + } + + qt_window_funnel_datetimev2 """select window_funnel(3600 * 3, 'deduplication', t.xwhen, t.xwhat = 1, t.xwhat = 2 ) AS level from windowfunnel_test_1 t; """ } From 1c3789e47998ea2ebbe445ddbb447f95a2b18c4a Mon Sep 17 00:00:00 2001 From: Gabriel Date: Mon, 25 Nov 2024 12:02:25 +0800 Subject: [PATCH 067/110] [fix](local shuffle) Fix bucket local shuffle (#44459) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Data in different buckets should be distributed into all tasks after bucket-hash local exchange. Before: ``` ┌─────────────────────────────────────────┐ │ ┌─────────┐ ┌───────────────────────┐ │ │ │Bucket 0 │ │ │ │ │ └─────────┘ │ │ │ │ │ LOCAL EXCHANGE SOURCE │ │ ┌──────► ┌─────────┐ │ (BUCKET HASH) │ │ │ │ │Bucket 1 │ │ │ │ │ │ └─────────┘ └───────────────────────┘ │ │ └─────────────────────────────────────────┘ ┌─────────────────────────────────────────────────────┐ │ │ ┌─────────┐ ┌───────┐ ┌─────────────────────┐ │ │ │ │Bucket 0 │ │ │ │ │ │ │ │ └─────────┘ │ │ │ │ │ │ │ │ SCAN │ │ LOCAL EXCHANGE SINK │ ├────────┤ │ ┌─────────┐ │ │ │ (BUCKET HASH) │ │ │ │ │Bucket 1 │ │ │ │ │ │ │ │ └─────────┘ └───────┘ └─────────────────────┘ │ │ └─────────────────────────────────────────────────────┘ │ ┌─────────────────────────────────────────┐ │ │ ┌───────────────────────┐ │ │ │ │ │ │ │ │ │ │ │ │ │ │ LOCAL EXCHANGE SOURCE │ │ └──────► │ (BUCKET HASH) │ │ │ │ │ │ │ └───────────────────────┘ │ └─────────────────────────────────────────┘ ``` After ``` ┌─────────────────────────────────────────┐ │ ┌─────────┐ ┌───────────────────────┐ │ │ │Bucket 0 │ │ │ │ │ └─────────┘ │ │ │ │ │ LOCAL EXCHANGE SOURCE │ │ ┌──────► │ (BUCKET HASH) │ │ │ │ │ │ │ │ │ └───────────────────────┘ │ │ └─────────────────────────────────────────┘ ┌─────────────────────────────────────────────────────┐ │ │ ┌─────────┐ ┌───────┐ ┌─────────────────────┐ │ │ │ │Bucket 0 │ │ │ │ │ │ │ │ └─────────┘ │ │ │ │ │ │ │ │ SCAN │ │ LOCAL EXCHANGE SINK │ ├────────┤ │ ┌─────────┐ │ │ │ (BUCKET HASH) │ │ │ │ │Bucket 1 │ │ │ │ │ │ │ │ └─────────┘ └───────┘ └─────────────────────┘ │ │ └─────────────────────────────────────────────────────┘ │ ┌─────────────────────────────────────────┐ │ │ ┌───────────────────────┐ │ │ │ │ │ │ │ │ ┌─────────┐ │ │ │ │ │ │Bucket 1 │ │ LOCAL EXCHANGE SOURCE │ │ └──────► └─────────┘ │ (BUCKET HASH) │ │ │ │ │ │ │ └───────────────────────┘ │ └─────────────────────────────────────────┘ ``` --- be/src/exprs/runtime_filter.cpp | 7 ++++--- .../java/org/apache/doris/qe/Coordinator.java | 14 ++++++-------- .../doris/qe/runtime/ThriftPlansBuilder.java | 15 ++++----------- gensrc/thrift/PaloInternalService.thrift | 4 ++-- 4 files changed, 16 insertions(+), 24 deletions(-) diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index 24333360ff62540..bac14b616b2ce65 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -1273,7 +1273,8 @@ void IRuntimeFilter::update_state() { // In pipelineX, runtime filters will be ready or timeout before open phase. if (expected == RuntimeFilterState::NOT_READY) { DCHECK(MonotonicMillis() - registration_time_ >= wait_times_ms); - COUNTER_SET(_wait_timer, MonotonicMillis() - registration_time_); + COUNTER_SET(_wait_timer, + int64_t((MonotonicMillis() - registration_time_) * NANOS_PER_MILLIS)); _rf_state_atomic = RuntimeFilterState::TIME_OUT; } } @@ -1292,7 +1293,7 @@ PrimitiveType IRuntimeFilter::column_type() const { void IRuntimeFilter::signal() { DCHECK(is_consumer()); - COUNTER_SET(_wait_timer, MonotonicMillis() - registration_time_); + COUNTER_SET(_wait_timer, int64_t((MonotonicMillis() - registration_time_) * NANOS_PER_MILLIS)); _rf_state_atomic.store(RuntimeFilterState::READY); if (!_filter_timer.empty()) { for (auto& timer : _filter_timer) { @@ -1539,7 +1540,7 @@ void IRuntimeFilter::init_profile(RuntimeProfile* parent_profile) { void IRuntimeFilter::update_runtime_filter_type_to_profile(uint64_t local_merge_time) { _profile->add_info_string("RealRuntimeFilterType", to_string(_wrapper->get_real_type())); _profile->add_info_string("LocalMergeTime", - std::to_string(local_merge_time / 1000000000.0) + " s"); + std::to_string((double)local_merge_time / NANOS_PER_SEC) + " s"); } std::string IRuntimeFilter::debug_string() const { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index acd0fbe0daeaaf9..e508efde42dbf66 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -1935,7 +1935,6 @@ protected void computeFragmentHosts() throws Exception { FInstanceExecParam instanceParam = new FInstanceExecParam(null, key, 0, params); instanceParam.perNodeScanRanges.put(planNodeId, scanRangeParams); - instanceParam.perNodeSharedScans.put(planNodeId, sharedScan); params.instanceExecParams.add(instanceParam); } params.ignoreDataDistribution = sharedScan; @@ -2757,13 +2756,11 @@ private void assignScanRanges(PlanFragmentId fragmentId, int parallelExecInstanc null, addressScanRange.getKey(), 0, params); for (Pair>> nodeScanRangeMap : scanRange) { - instanceParam.addBucketSeq(nodeScanRangeMap.first); for (Map.Entry> nodeScanRange : nodeScanRangeMap.second.entrySet()) { if (!instanceParam.perNodeScanRanges.containsKey(nodeScanRange.getKey())) { range.put(nodeScanRange.getKey(), Lists.newArrayList()); instanceParam.perNodeScanRanges.put(nodeScanRange.getKey(), Lists.newArrayList()); - instanceParam.perNodeSharedScans.put(nodeScanRange.getKey(), true); } range.get(nodeScanRange.getKey()).addAll(nodeScanRange.getValue()); instanceParam.perNodeScanRanges.get(nodeScanRange.getKey()) @@ -2775,6 +2772,12 @@ private void assignScanRanges(PlanFragmentId fragmentId, int parallelExecInstanc params.instanceExecParams.add(new FInstanceExecParam( null, addressScanRange.getKey(), 0, params)); } + int index = 0; + for (Pair>> nodeScanRangeMap : scanRange) { + params.instanceExecParams.get(index % params.instanceExecParams.size()) + .addBucketSeq(nodeScanRangeMap.first); + index++; + } } else { int expectedInstanceNum = 1; if (parallelExecInstanceNum > 1) { @@ -3131,10 +3134,8 @@ Map toThrift(int backendNum) { for (int i = 0; i < instanceExecParams.size(); ++i) { final FInstanceExecParam instanceExecParam = instanceExecParams.get(i); Map> scanRanges = instanceExecParam.perNodeScanRanges; - Map perNodeSharedScans = instanceExecParam.perNodeSharedScans; if (scanRanges == null) { scanRanges = Maps.newHashMap(); - perNodeSharedScans = Maps.newHashMap(); } if (!res.containsKey(instanceExecParam.host)) { TPipelineFragmentParams params = new TPipelineFragmentParams(); @@ -3162,7 +3163,6 @@ Map toThrift(int backendNum) { params.setFileScanParams(fileScanRangeParamsMap); params.setNumBuckets(fragment.getBucketNum()); - params.setPerNodeSharedScans(perNodeSharedScans); params.setTotalInstances(instanceExecParams.size()); if (ignoreDataDistribution) { params.setParallelInstances(parallelTasksNum); @@ -3187,7 +3187,6 @@ Map toThrift(int backendNum) { localParams.setFragmentInstanceId(instanceExecParam.instanceId); localParams.setPerNodeScanRanges(scanRanges); - localParams.setPerNodeSharedScans(perNodeSharedScans); localParams.setSenderId(i); localParams.setBackendNum(backendNum++); localParams.setRuntimeFilterParams(new TRuntimeFilterParams()); @@ -3335,7 +3334,6 @@ static class FInstanceExecParam { TUniqueId instanceId; TNetworkAddress host; Map> perNodeScanRanges = Maps.newHashMap(); - Map perNodeSharedScans = Maps.newHashMap(); int perFragmentInstanceIdx; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java index f0e3febe1928546..a02ee90e901cd52 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java @@ -423,7 +423,7 @@ private static void setScanSourceParam( boolean isLocalShuffle = instance instanceof LocalShuffleAssignedJob; if (isLocalShuffle && ((LocalShuffleAssignedJob) instance).receiveDataFromLocal) { - // save thrift rpc message size, don't need perNodeScanRanges and perNodeSharedScans, + // save thrift rpc message size, don't need perNodeScanRanges, // but the perNodeScanRanges is required rpc field instanceParams.setPerNodeScanRanges(Maps.newLinkedHashMap()); return; @@ -459,19 +459,16 @@ private static void ignoreDataDistribution(TPipelineFragmentParams currentFragme private static PerNodeScanParams computeDefaultScanSourceParam(DefaultScanSource defaultScanSource) { Map> perNodeScanRanges = Maps.newLinkedHashMap(); - Map perNodeSharedScans = Maps.newLinkedHashMap(); for (Entry kv : defaultScanSource.scanNodeToScanRanges.entrySet()) { int scanNodeId = kv.getKey().getId().asInt(); perNodeScanRanges.put(scanNodeId, kv.getValue().params); - perNodeSharedScans.put(scanNodeId, true); } - return new PerNodeScanParams(perNodeScanRanges, perNodeSharedScans); + return new PerNodeScanParams(perNodeScanRanges); } private static PerNodeScanParams computeBucketScanSourceParam(BucketScanSource bucketScanSource) { Map> perNodeScanRanges = Maps.newLinkedHashMap(); - Map perNodeSharedScans = Maps.newLinkedHashMap(); for (Entry> kv : bucketScanSource.bucketIndexToScanNodeToTablets.entrySet()) { Map scanNodeToRanges = kv.getValue(); @@ -479,10 +476,9 @@ private static PerNodeScanParams computeBucketScanSourceParam(BucketScanSource b int scanNodeId = kv2.getKey().getId().asInt(); List scanRanges = perNodeScanRanges.computeIfAbsent(scanNodeId, ArrayList::new); scanRanges.addAll(kv2.getValue().params); - perNodeSharedScans.put(scanNodeId, true); } } - return new PerNodeScanParams(perNodeScanRanges, perNodeSharedScans); + return new PerNodeScanParams(perNodeScanRanges); } private static Map computeBucketIdToInstanceId( @@ -562,12 +558,9 @@ private static void filterInstancesWhichReceiveDataFromRemote( private static class PerNodeScanParams { Map> perNodeScanRanges; - Map perNodeSharedScans; - public PerNodeScanParams(Map> perNodeScanRanges, - Map perNodeSharedScans) { + public PerNodeScanParams(Map> perNodeScanRanges) { this.perNodeScanRanges = perNodeScanRanges; - this.perNodeSharedScans = perNodeSharedScans; } } } diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index 392aa8658df1d2a..9a0fd910d943879 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -776,7 +776,7 @@ struct TPipelineInstanceParams { 4: optional i32 sender_id 5: optional TRuntimeFilterParams runtime_filter_params 6: optional i32 backend_num - 7: optional map per_node_shared_scans + 7: optional map per_node_shared_scans // deprecated 8: optional list topn_filter_source_node_ids // deprecated after we set topn_filter_descs 9: optional list topn_filter_descs } @@ -820,7 +820,7 @@ struct TPipelineFragmentParams { 33: optional i32 num_local_sink 34: optional i32 num_buckets 35: optional map bucket_seq_to_instance_idx - 36: optional map per_node_shared_scans + 36: optional map per_node_shared_scans // deprecated 37: optional i32 parallel_instances 38: optional i32 total_instances 39: optional map shuffle_idx_to_instance_idx From 647cd86c4126808f4e5d458f6d6f3fae8f994a26 Mon Sep 17 00:00:00 2001 From: feifeifeimoon Date: Mon, 25 Nov 2024 13:11:53 +0800 Subject: [PATCH 068/110] [fix](tools) tpcds-tools: fix TPCDS_DBGEN_DIR (#44409) ### What problem does this PR solve? Issue Number: close #xxx Related PR: #43668 Problem Summary: This associated PR modified the TPCDD_DBGEN-DIR path, but omitted the file 'create tpcs-tables. sh' ```bash bin/gen-tpcds-data.sh -s 100 Scale Factor: 100 Parallelism: 10 /test/doris/tools/tpcds-tools/bin/DSGen-software-code-3.2.0rc1/tools/dsdgen does not exist. Run build-tpcds-dsdgen.sh first to build it first. ``` --- tools/tpcds-tools/bin/build-tpcds-tools.sh | 4 ++-- tools/tpcds-tools/bin/gen-tpcds-data.sh | 2 +- tools/tpcds-tools/bin/gen-tpcds-queries.sh | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tools/tpcds-tools/bin/build-tpcds-tools.sh b/tools/tpcds-tools/bin/build-tpcds-tools.sh index e5963cd8e8d75d1..b8f1b2a8fa5a56b 100755 --- a/tools/tpcds-tools/bin/build-tpcds-tools.sh +++ b/tools/tpcds-tools/bin/build-tpcds-tools.sh @@ -46,8 +46,8 @@ check_prerequest() { check_prerequest "unzip -h" "unzip" # download tpcds tools package first -if [[ -d "${CURDIR}/DSGen-software-code-3.2.0rc1" ]]; then - echo "If you want to rebuild TPC-DS_Tools_v3.2.0 again, please delete ${CURDIR}/DSGen-software-code-3.2.0rc1 first." +if [[ -d "${CURDIR}/DSGen-software-code-3.2.0rc2" ]]; then + echo "If you want to rebuild TPC-DS_Tools_v3.2.0 again, please delete ${CURDIR}/DSGen-software-code-3.2.0rc2 first." elif [[ -f "TPC-DS_Tools_v3.2.0rc2.zip" ]]; then unzip TPC-DS_Tools_v3.2.0rc2.zip -d "${CURDIR}/" else diff --git a/tools/tpcds-tools/bin/gen-tpcds-data.sh b/tools/tpcds-tools/bin/gen-tpcds-data.sh index ebd5a3f694dddac..21cbacf2ce49625 100755 --- a/tools/tpcds-tools/bin/gen-tpcds-data.sh +++ b/tools/tpcds-tools/bin/gen-tpcds-data.sh @@ -29,7 +29,7 @@ ROOT=$( ) CURDIR="${ROOT}" -TPCDS_DBGEN_DIR="${CURDIR}/DSGen-software-code-3.2.0rc1/tools" +TPCDS_DBGEN_DIR="${CURDIR}/DSGen-software-code-3.2.0rc2/tools" usage() { echo " diff --git a/tools/tpcds-tools/bin/gen-tpcds-queries.sh b/tools/tpcds-tools/bin/gen-tpcds-queries.sh index eb4f292606d4c26..cd5542bf5193efd 100755 --- a/tools/tpcds-tools/bin/gen-tpcds-queries.sh +++ b/tools/tpcds-tools/bin/gen-tpcds-queries.sh @@ -29,7 +29,7 @@ ROOT=$( ) CURDIR="${ROOT}" -TPCDS_DSQGEN_DIR="${CURDIR}/DSGen-software-code-3.2.0rc1/tools" +TPCDS_DSQGEN_DIR="${CURDIR}/DSGen-software-code-3.2.0rc2/tools" TPCDS_QUERIE_DIR="${CURDIR}/../queries" usage() { From 71b7d3f4ae2df583c821f8102a203296983162c7 Mon Sep 17 00:00:00 2001 From: "Mingyu Chen (Rayner)" Date: Mon, 25 Nov 2024 14:29:20 +0800 Subject: [PATCH 069/110] [fix](readme) remove contributor graph and change to contributor wall (#44534) ### What problem does this PR solve? Issue Number: close #44197 --- README.md | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/README.md b/README.md index c999651ddee68df..94f9f4b777f8f50 100644 --- a/README.md +++ b/README.md @@ -177,15 +177,10 @@ In terms of optimizers, Doris uses a combination of CBO and RBO. RBO supports co **Apache Doris has graduated from Apache incubator successfully and become a Top-Level Project in June 2022**. -Currently, the Apache Doris community has gathered more than 400 contributors from nearly 200 companies in different industries, and the number of active contributors is close to 100 per month. - - -[![Monthly Active Contributors](https://contributor-overtime-api.apiseven.com/contributors-svg?chart=contributorMonthlyActivity&repo=apache/doris)](https://www.apiseven.com/en/contributor-graph?chart=contributorMonthlyActivity&repo=apache/doris) - -[![Contributor over time](https://contributor-overtime-api.apiseven.com/contributors-svg?chart=contributorOverTime&repo=apache/doris)](https://www.apiseven.com/en/contributor-graph?chart=contributorOverTime&repo=apache/doris) - We deeply appreciate 🔗[community contributors](https://github.com/apache/doris/graphs/contributors) for their contribution to Apache Doris. +[![contrib graph](https://contrib.rocks/image?repo=apache/doris)](https://github.com/apache/doris/graphs/contributors) + ## 👨‍👩‍👧‍👦 Users Apache Doris now has a wide user base in China and around the world, and as of today, **Apache Doris is used in production environments in thousands of companies worldwide.** More than 80% of the top 50 Internet companies in China in terms of market capitalization or valuation have been using Apache Doris for a long time, including Baidu, Meituan, Xiaomi, Jingdong, Bytedance, Tencent, NetEase, Kwai, Sina, 360, Mihoyo, and Ke Holdings. It is also widely used in some traditional industries such as finance, energy, manufacturing, and telecommunications. From 8a8d37c271c9ba2031ee6dcd024940fb669c04ef Mon Sep 17 00:00:00 2001 From: Mryange Date: Mon, 25 Nov 2024 15:34:47 +0800 Subject: [PATCH 070/110] [refine](column) ColumnArray does not implement the insert data function. (#43926) ColumnArray cannot determine how many elements it contains from a single block of memory. The original approach where the result of get data at cannot be inserted back into insert data. Therefore, this function is not implemented directly. --- be/src/vec/columns/column.h | 11 ---- be/src/vec/columns/column_array.cpp | 52 +++---------------- be/src/vec/columns/column_complex.h | 3 -- be/src/vec/columns/column_const.cpp | 33 ------------ be/src/vec/columns/column_const.h | 10 ---- be/src/vec/columns/column_decimal.h | 2 - be/src/vec/columns/column_dictionary.h | 4 -- be/src/vec/columns/column_nullable.h | 6 --- be/src/vec/columns/column_object.h | 5 -- be/src/vec/columns/column_vector.h | 2 - be/src/vec/columns/predicate_column.h | 3 -- be/test/vec/columns/column_hash_func_test.cpp | 6 +-- 12 files changed, 9 insertions(+), 128 deletions(-) diff --git a/be/src/vec/columns/column.h b/be/src/vec/columns/column.h index 19462b182bd2fd5..96408579a843c4b 100644 --- a/be/src/vec/columns/column.h +++ b/be/src/vec/columns/column.h @@ -601,23 +601,12 @@ class IColumn : public COW { * To avoid confusion between these cases, we don't have isContiguous method. */ - /// Values in column are represented as continuous memory segment of fixed size. Implies values_have_fixed_size. - virtual bool is_fixed_and_contiguous() const { return false; } - - /// If is_fixed_and_contiguous, returns the underlying data array, otherwise throws an exception. virtual StringRef get_raw_data() const { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "Column {} is not a contiguous block of memory", get_name()); return StringRef {}; } - /// If values_have_fixed_size, returns size of value, otherwise throw an exception. - virtual size_t size_of_value_if_fixed() const { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "Values of column {} are not fixed size.", get_name()); - return 0; - } - /// Returns ratio of values in column, that are equal to default value of column. /// Checks only @sample_ratio ratio of rows. virtual double get_ratio_of_default_rows(double sample_ratio = 1.0) const { return 0.0; } diff --git a/be/src/vec/columns/column_array.cpp b/be/src/vec/columns/column_array.cpp index bd4464e2caf81c9..0c5a53cdb24ca55 100644 --- a/be/src/vec/columns/column_array.cpp +++ b/be/src/vec/columns/column_array.cpp @@ -151,26 +151,13 @@ void ColumnArray::get(size_t n, Field& res) const { } StringRef ColumnArray::get_data_at(size_t n) const { - /** Returns the range of memory that covers all elements of the array. - * Works for arrays of fixed length values. - * For arrays of strings and arrays of arrays, the resulting chunk of memory may not be one-to-one correspondence with the elements, - * since it contains only the data laid in succession, but not the offsets. - */ - size_t offset_of_first_elem = offset_at(n); - StringRef first; - if (offset_of_first_elem < get_data().size()) { - first = get_data().get_data_at(offset_of_first_elem); - } - - size_t array_size = size_at(n); - if (array_size == 0) { - return StringRef(first.data, 0); - } - - size_t offset_of_last_elem = offset_at(n + 1) - 1; - StringRef last = get_data().get_data_at(offset_of_last_elem); + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "Method get_data_at is not supported for " + get_name()); +} - return StringRef(first.data, last.data + last.size - first.data); +void ColumnArray::insert_data(const char* pos, size_t length) { + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "Method insert_data is not supported for " + get_name()); } bool ColumnArray::is_default_at(size_t n) const { @@ -178,33 +165,6 @@ bool ColumnArray::is_default_at(size_t n) const { return offsets_data[n] == offsets_data[static_cast(n) - 1]; } -void ColumnArray::insert_data(const char* pos, size_t length) { - /** Similarly - only for arrays of fixed length values. - */ - if (!data->is_fixed_and_contiguous()) { - throw doris::Exception(ErrorCode::INTERNAL_ERROR, - "Method insert_data should have_fixed_size, {} is not suitable", - get_name()); - } - - size_t field_size = data->size_of_value_if_fixed(); - - size_t elems = 0; - - if (length) { - const char* end = pos + length; - for (; pos + field_size <= end; pos += field_size, ++elems) - data->insert_data(pos, field_size); - - if (pos != end) - throw doris::Exception(ErrorCode::INTERNAL_ERROR, - "Incorrect length argument for method ColumnArray::insert_data"); - __builtin_unreachable(); - } - - get_offsets().push_back(get_offsets().back() + elems); -} - StringRef ColumnArray::serialize_value_into_arena(size_t n, Arena& arena, char const*& begin) const { size_t array_size = size_at(n); diff --git a/be/src/vec/columns/column_complex.h b/be/src/vec/columns/column_complex.h index 24b6b7ddbd7b549..14ae940c9d7ba07 100644 --- a/be/src/vec/columns/column_complex.h +++ b/be/src/vec/columns/column_complex.h @@ -207,9 +207,6 @@ class ColumnComplexType final : public COWHelper> // TODO add hash function } - bool is_fixed_and_contiguous() const override { return true; } - size_t size_of_value_if_fixed() const override { return sizeof(T); } - StringRef get_raw_data() const override { return StringRef(reinterpret_cast(data.data()), data.size()); } diff --git a/be/src/vec/columns/column_const.cpp b/be/src/vec/columns/column_const.cpp index a4b3127ad6cac96..f751f1d8d3e1b87 100644 --- a/be/src/vec/columns/column_const.cpp +++ b/be/src/vec/columns/column_const.cpp @@ -110,39 +110,6 @@ ColumnPtr ColumnConst::permute(const Permutation& perm, size_t limit) const { return ColumnConst::create(data, limit); } -void ColumnConst::update_crcs_with_value(uint32_t* __restrict hashes, doris::PrimitiveType type, - uint32_t rows, uint32_t offset, - const uint8_t* __restrict null_data) const { - DCHECK(null_data == nullptr); - DCHECK(rows == size()); - auto real_data = data->get_data_at(0); - if (real_data.data == nullptr) { - for (int i = 0; i < rows; ++i) { - hashes[i] = HashUtil::zlib_crc_hash_null(hashes[i]); - } - } else { - for (int i = 0; i < rows; ++i) { - hashes[i] = RawValue::zlib_crc32(real_data.data, real_data.size, type, hashes[i]); - } - } -} - -void ColumnConst::update_hashes_with_value(uint64_t* __restrict hashes, - const uint8_t* __restrict null_data) const { - DCHECK(null_data == nullptr); - auto real_data = data->get_data_at(0); - auto real_size = size(); - if (real_data.data == nullptr) { - for (int i = 0; i < real_size; ++i) { - hashes[i] = HashUtil::xxHash64NullWithSeed(hashes[i]); - } - } else { - for (int i = 0; i < real_size; ++i) { - hashes[i] = HashUtil::xxHash64WithSeed(real_data.data, real_data.size, hashes[i]); - } - } -} - void ColumnConst::get_permutation(bool /*reverse*/, size_t /*limit*/, int /*nan_direction_hint*/, Permutation& res) const { res.resize(s); diff --git a/be/src/vec/columns/column_const.h b/be/src/vec/columns/column_const.h index 980d9d64148ae44..ee3860f0635c932 100644 --- a/be/src/vec/columns/column_const.h +++ b/be/src/vec/columns/column_const.h @@ -208,14 +208,6 @@ class ColumnConst final : public COWHelper { data->update_hash_with_value(0, hash); } - // (TODO.Amory) here may not use column_const update hash, and PrimitiveType is not used. - void update_crcs_with_value(uint32_t* __restrict hashes, PrimitiveType type, uint32_t rows, - uint32_t offset = 0, - const uint8_t* __restrict null_data = nullptr) const override; - - void update_hashes_with_value(uint64_t* __restrict hashes, - const uint8_t* __restrict null_data) const override; - ColumnPtr filter(const Filter& filt, ssize_t result_size_hint) const override; size_t filter(const Filter& filter) override; @@ -263,8 +255,6 @@ class ColumnConst final : public COWHelper { bool is_concrete_nullable() const override { return is_column_nullable(*data); } bool only_null() const override { return data->is_null_at(0); } bool is_numeric() const override { return data->is_numeric(); } - bool is_fixed_and_contiguous() const override { return data->is_fixed_and_contiguous(); } - size_t size_of_value_if_fixed() const override { return data->size_of_value_if_fixed(); } StringRef get_raw_data() const override { return data->get_raw_data(); } /// Not part of the common interface. diff --git a/be/src/vec/columns/column_decimal.h b/be/src/vec/columns/column_decimal.h index d754831cc56a86e..4c2f69d5ef3fb66 100644 --- a/be/src/vec/columns/column_decimal.h +++ b/be/src/vec/columns/column_decimal.h @@ -106,8 +106,6 @@ class ColumnDecimal final : public COWHelper> { bool is_numeric() const override { return false; } bool is_column_decimal() const override { return true; } - bool is_fixed_and_contiguous() const override { return true; } - size_t size_of_value_if_fixed() const override { return sizeof(T); } size_t size() const override { return data.size(); } size_t byte_size() const override { return data.size() * sizeof(data[0]); } diff --git a/be/src/vec/columns/column_dictionary.h b/be/src/vec/columns/column_dictionary.h index 69e04973af77a19..ae7d001a31d78cc 100644 --- a/be/src/vec/columns/column_dictionary.h +++ b/be/src/vec/columns/column_dictionary.h @@ -158,10 +158,6 @@ class ColumnDictionary final : public COWHelper> { __builtin_unreachable(); } - bool is_fixed_and_contiguous() const override { return true; } - - size_t size_of_value_if_fixed() const override { return sizeof(T); } - [[noreturn]] StringRef get_raw_data() const override { throw doris::Exception(ErrorCode::INTERNAL_ERROR, "get_raw_data not supported in ColumnDictionary"); diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h index 2b87aa982cae501..252144fbc5f6128 100644 --- a/be/src/vec/columns/column_nullable.h +++ b/be/src/vec/columns/column_nullable.h @@ -334,18 +334,12 @@ class ColumnNullable final : public COWHelper, public N bool is_column_array() const override { return get_nested_column().is_column_array(); } bool is_column_map() const override { return get_nested_column().is_column_map(); } bool is_column_struct() const override { return get_nested_column().is_column_struct(); } - bool is_fixed_and_contiguous() const override { return false; } bool is_exclusive() const override { return IColumn::is_exclusive() && nested_column->is_exclusive() && get_null_map_column().is_exclusive(); } - size_t size_of_value_if_fixed() const override { - return get_null_map_column().size_of_value_if_fixed() + - nested_column->size_of_value_if_fixed(); - } - bool only_null() const override { return size() == 1 && is_null_at(0); } // used in schema change diff --git a/be/src/vec/columns/column_object.h b/be/src/vec/columns/column_object.h index 1c8f38056c9d546..21bb44691151b08 100644 --- a/be/src/vec/columns/column_object.h +++ b/be/src/vec/columns/column_object.h @@ -525,11 +525,6 @@ class ColumnObject final : public COWHelper { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "get_raw_data" + get_name()); } - size_t size_of_value_if_fixed() const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "size_of_value_if_fixed" + get_name()); - } - StringRef get_data_at(size_t) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "get_data_at" + get_name()); } diff --git a/be/src/vec/columns/column_vector.h b/be/src/vec/columns/column_vector.h index 2676d6d344468b0..2cb320b69920956 100644 --- a/be/src/vec/columns/column_vector.h +++ b/be/src/vec/columns/column_vector.h @@ -373,8 +373,6 @@ class ColumnVector final : public COWHelper> { ColumnPtr replicate(const IColumn::Offsets& offsets) const override; - bool is_fixed_and_contiguous() const override { return true; } - size_t size_of_value_if_fixed() const override { return sizeof(T); } StringRef get_raw_data() const override { return StringRef(reinterpret_cast(data.data()), data.size()); } diff --git a/be/src/vec/columns/predicate_column.h b/be/src/vec/columns/predicate_column.h index c2c6456d86255bb..7e15656fe1d2043 100644 --- a/be/src/vec/columns/predicate_column.h +++ b/be/src/vec/columns/predicate_column.h @@ -376,9 +376,6 @@ class PredicateColumnType final : public COWHelper(d); ColumnPtr col_a = a->create_column_const_with_default_value(1); // xxHash - EXPECT_NO_FATAL_FAILURE(col_a->update_hashes_with_value(xx_hashes)); + EXPECT_NO_FATAL_FAILURE(unpack_if_const(col_a).first->update_hashes_with_value(xx_hashes)); std::cout << xx_hashes[0] << std::endl; // crcHash - EXPECT_NO_FATAL_FAILURE( - col_a->update_crcs_with_value(crc_hashes, PrimitiveType::TYPE_ARRAY, 1)); + EXPECT_NO_FATAL_FAILURE(unpack_if_const(col_a).first->update_crcs_with_value( + crc_hashes, PrimitiveType::TYPE_ARRAY, 1)); std::cout << crc_hashes[0] << std::endl; } } From f1021783f299f24dffb248a7a206db7c5a763908 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Mon, 25 Nov 2024 16:05:35 +0800 Subject: [PATCH 071/110] [fix](mtmv) NPE in rebuild MTMV info when cannot find catalog name from ID (#44464) The ID stored in the previous version and the name stored in the new version need to be checked for compatibility. If the catalog is deleted or rebuilt, compatibility will fail, but null pointers should not be reported --- .../java/org/apache/doris/mtmv/BaseTableInfo.java | 9 +++++++++ .../main/java/org/apache/doris/mtmv/MTMVUtil.java | 3 +-- .../mv/InitMaterializationContextHook.java | 12 ++++++++++-- 3 files changed, 20 insertions(+), 4 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseTableInfo.java index 48796c3360773cb..fcf18d73a269bb5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseTableInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseTableInfo.java @@ -110,6 +110,15 @@ public void setTableName(String tableName) { this.tableName = tableName; } + // if compatible failed due catalog dropped, ctlName will be null + public boolean isInternalTable() { + if (!StringUtils.isEmpty(ctlName)) { + return InternalCatalog.INTERNAL_CATALOG_NAME.equals(ctlName); + } else { + return InternalCatalog.INTERNAL_CATALOG_ID == ctlId; + } + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVUtil.java index e84136489291f5d..256707fcbd9e4c3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVUtil.java @@ -28,7 +28,6 @@ import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.CatalogMgr; -import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.executable.DateTimeExtractAndTransform; import org.apache.doris.nereids.trees.expressions.literal.DateTimeV2Literal; @@ -96,7 +95,7 @@ public static MTMV getMTMV(long dbId, long mtmvId) throws DdlException, MetaNotF public static boolean mtmvContainsExternalTable(MTMV mtmv) { Set baseTables = mtmv.getRelation().getBaseTablesOneLevel(); for (BaseTableInfo baseTableInfo : baseTables) { - if (!baseTableInfo.getCtlName().equals(InternalCatalog.INTERNAL_CATALOG_NAME)) { + if (!baseTableInfo.isInternalTable()) { return true; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java index 0ad6f8f74a44109..2e8baecf1656390 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java @@ -41,6 +41,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; +import org.apache.commons.collections.CollectionUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -127,8 +128,15 @@ protected Set getAvailableMTMVs(Set usedTables, CascadesContext c private List createAsyncMaterializationContext(CascadesContext cascadesContext, Set usedTables) { - Set availableMTMVs = getAvailableMTMVs(usedTables, cascadesContext); - if (availableMTMVs.isEmpty()) { + Set availableMTMVs; + try { + availableMTMVs = getAvailableMTMVs(usedTables, cascadesContext); + } catch (Exception e) { + LOG.warn(String.format("MaterializationContext getAvailableMTMVs generate fail, current queryId is %s", + cascadesContext.getConnectContext().getQueryIdentifier()), e); + return ImmutableList.of(); + } + if (CollectionUtils.isEmpty(availableMTMVs)) { LOG.debug("Enable materialized view rewrite but availableMTMVs is empty, current queryId " + "is {}", cascadesContext.getConnectContext().getQueryIdentifier()); return ImmutableList.of(); From 3c8d75250176536abd8cba987a55eda9002dbee3 Mon Sep 17 00:00:00 2001 From: wuwenchi Date: Mon, 25 Nov 2024 16:16:51 +0800 Subject: [PATCH 072/110] [opt](paimon)Optimize the storage location of the serialized paimon table (#44274) ### What problem does this PR solve? Related PR: #43167 Previously, the serialized paimon table was placed in `TPaimonFileDesc`, which resulted in each split storing a serialized paimon table. It will wasted a lot of space if a table has lots of split. Now change the storage location of the serialized paimon table so that all splits of a table can share the data. --- be/src/vec/exec/format/table/paimon_jni_reader.cpp | 7 ++++--- be/src/vec/exec/format/table/paimon_jni_reader.h | 3 ++- be/src/vec/exec/scan/vfile_scanner.cpp | 4 ++-- .../java/org/apache/doris/paimon/PaimonJniScanner.java | 4 ++-- .../org/apache/doris/datasource/FileQueryScanNode.java | 8 ++++++++ .../doris/datasource/paimon/source/PaimonScanNode.java | 7 +++++++ gensrc/thrift/PlanNodes.thrift | 7 ++++++- 7 files changed, 31 insertions(+), 9 deletions(-) diff --git a/be/src/vec/exec/format/table/paimon_jni_reader.cpp b/be/src/vec/exec/format/table/paimon_jni_reader.cpp index 30358eace1aae2b..83705426e890ecb 100644 --- a/be/src/vec/exec/format/table/paimon_jni_reader.cpp +++ b/be/src/vec/exec/format/table/paimon_jni_reader.cpp @@ -40,7 +40,8 @@ const std::string PaimonJniReader::HADOOP_OPTION_PREFIX = "hadoop."; PaimonJniReader::PaimonJniReader(const std::vector& file_slot_descs, RuntimeState* state, RuntimeProfile* profile, - const TFileRangeDesc& range) + const TFileRangeDesc& range, + const TFileScanRangeParams* range_params) : JniReader(file_slot_descs, state, profile) { std::vector column_names; std::vector column_types; @@ -61,8 +62,8 @@ PaimonJniReader::PaimonJniReader(const std::vector& file_slot_d std::to_string(range.table_format_params.paimon_params.last_update_time); params["required_fields"] = join(column_names, ","); params["columns_types"] = join(column_types, "#"); - if (range.table_format_params.paimon_params.__isset.paimon_table) { - params["paimon_table"] = range.table_format_params.paimon_params.paimon_table; + if (range_params->__isset.serialized_table) { + params["serialized_table"] = range_params->serialized_table; } // Used to create paimon option diff --git a/be/src/vec/exec/format/table/paimon_jni_reader.h b/be/src/vec/exec/format/table/paimon_jni_reader.h index 6ecf6cd1f153e1b..220a6f1f2e9665e 100644 --- a/be/src/vec/exec/format/table/paimon_jni_reader.h +++ b/be/src/vec/exec/format/table/paimon_jni_reader.h @@ -53,7 +53,8 @@ class PaimonJniReader : public JniReader { static const std::string PAIMON_OPTION_PREFIX; static const std::string HADOOP_OPTION_PREFIX; PaimonJniReader(const std::vector& file_slot_descs, RuntimeState* state, - RuntimeProfile* profile, const TFileRangeDesc& range); + RuntimeProfile* profile, const TFileRangeDesc& range, + const TFileScanRangeParams* range_params); ~PaimonJniReader() override = default; diff --git a/be/src/vec/exec/scan/vfile_scanner.cpp b/be/src/vec/exec/scan/vfile_scanner.cpp index 9353887799207dd..982ac6072dc0186 100644 --- a/be/src/vec/exec/scan/vfile_scanner.cpp +++ b/be/src/vec/exec/scan/vfile_scanner.cpp @@ -793,8 +793,8 @@ Status VFileScanner::_get_next_reader() { _cur_reader = std::move(mc_reader); } else if (range.__isset.table_format_params && range.table_format_params.table_format_type == "paimon") { - _cur_reader = - PaimonJniReader::create_unique(_file_slot_descs, _state, _profile, range); + _cur_reader = PaimonJniReader::create_unique(_file_slot_descs, _state, _profile, + range, _params); init_status = ((PaimonJniReader*)(_cur_reader.get())) ->init_reader(_colname_to_value_range); } else if (range.__isset.table_format_params && diff --git a/fe/be-java-extensions/paimon-scanner/src/main/java/org/apache/doris/paimon/PaimonJniScanner.java b/fe/be-java-extensions/paimon-scanner/src/main/java/org/apache/doris/paimon/PaimonJniScanner.java index e85d465f66308b9..6ffd5f1ad9029ab 100644 --- a/fe/be-java-extensions/paimon-scanner/src/main/java/org/apache/doris/paimon/PaimonJniScanner.java +++ b/fe/be-java-extensions/paimon-scanner/src/main/java/org/apache/doris/paimon/PaimonJniScanner.java @@ -224,8 +224,8 @@ protected TableSchema parseTableSchema() throws UnsupportedOperationException { } private void initTable() { - if (params.containsKey("paimon_table")) { - table = PaimonUtils.deserialize(params.get("paimon_table")); + if (params.containsKey("serialized_table")) { + table = PaimonUtils.deserialize(params.get("serialized_table")); } else { PaimonTableCacheKey key = new PaimonTableCacheKey(ctlId, dbId, tblId, paimonOptionParams, hadoopOptionParams, dbName, tblName); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java index 96dae05b7fd0459..3107b4ed5d7b857 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java @@ -76,6 +76,7 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.Optional; /** * FileQueryScanNode for querying the file access type of catalog, now only support @@ -261,6 +262,11 @@ public TFileScanRangeParams getFileScanRangeParams() { protected void setScanParams(TFileRangeDesc rangeDesc, Split split) { } + // Serialize the table to be scanned to BE's jni reader + protected Optional getSerializedTable() { + return Optional.empty(); + } + @Override public void createScanRangeLocations() throws UserException { long start = System.currentTimeMillis(); @@ -369,6 +375,8 @@ public void createScanRangeLocations() throws UserException { } } + getSerializedTable().ifPresent(params::setSerializedTable); + if (ConnectContext.get().getExecutor() != null) { ConnectContext.get().getExecutor().getSummaryProfile().setCreateScanRangeFinishTime(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java index b61964481daac98..59f51c8425c7f2b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java @@ -102,6 +102,7 @@ public String toString() { private int paimonSplitNum = 0; private List splitStats = new ArrayList<>(); private SessionVariable sessionVariable; + private String serializedTable; public PaimonScanNode(PlanNodeId id, TupleDescriptor desc, @@ -115,6 +116,7 @@ public PaimonScanNode(PlanNodeId id, protected void doInitialize() throws UserException { super.doInitialize(); source = new PaimonSource(desc); + serializedTable = encodeObjectToString(source.getPaimonTable()); Preconditions.checkNotNull(source); } @@ -144,6 +146,11 @@ protected void setScanParams(TFileRangeDesc rangeDesc, Split split) { } } + @Override + protected Optional getSerializedTable() { + return Optional.of(serializedTable); + } + private void setPaimonParams(TFileRangeDesc rangeDesc, PaimonSplit paimonSplit) { TTableFormatFileDesc tableFormatFileDesc = new TTableFormatFileDesc(); tableFormatFileDesc.setTableFormatType(paimonSplit.getTableFormatType().value()); diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index ec4497b267b22f0..0bbd364fda1c2a0 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -332,7 +332,7 @@ struct TPaimonFileDesc { 11: optional string file_format 12: optional TPaimonDeletionFileDesc deletion_file; 13: optional map hadoop_conf // deprecated - 14: optional string paimon_table + 14: optional string paimon_table // deprecated } struct TTrinoConnectorFileDesc { @@ -448,6 +448,11 @@ struct TFileScanRangeParams { 22: optional TTextSerdeType text_serde_type // used by flexible partial update 23: optional string sequence_map_col + // table from FE, used for jni scanner + // BE can use table director: + // 1. Reduce the access to HMS and HDFS on the JNI side. + // 2. There will be no inconsistency between the fe and be tables. + 24: optional string serialized_table } struct TFileRangeDesc { From 8b68d08124307f057d2c2aff1c6bca041f610026 Mon Sep 17 00:00:00 2001 From: zzzxl Date: Mon, 25 Nov 2024 17:32:42 +0800 Subject: [PATCH 073/110] [fix](inverted index) Content Check for Tokenize Function Parser (#44465) Problem Summary: 1. Prevent users from mistakenly assuming other tokenizers exist. --- be/src/vec/functions/function_tokenize.cpp | 6 ++++++ .../suites/inverted_index_p0/test_tokenize.groovy | 11 +++++++++++ 2 files changed, 17 insertions(+) diff --git a/be/src/vec/functions/function_tokenize.cpp b/be/src/vec/functions/function_tokenize.cpp index b8e1565ebb48221..0bcd31af40dac79 100644 --- a/be/src/vec/functions/function_tokenize.cpp +++ b/be/src/vec/functions/function_tokenize.cpp @@ -142,6 +142,12 @@ Status FunctionTokenize::execute_impl(FunctionContext* /*context*/, Block& block } inverted_index_ctx.parser_type = get_inverted_index_parser_type_from_string( get_parser_string_from_properties(properties)); + if (inverted_index_ctx.parser_type == InvertedIndexParserType::PARSER_UNKNOWN) { + return Status::Error( + "unsupported parser type. currently, only 'english', 'chinese', and " + "'unicode' analyzers are supported."); + } + inverted_index_ctx.parser_mode = get_parser_mode_string_from_properties(properties); inverted_index_ctx.char_filter_map = get_parser_char_filter_map_from_properties(properties); diff --git a/regression-test/suites/inverted_index_p0/test_tokenize.groovy b/regression-test/suites/inverted_index_p0/test_tokenize.groovy index 8d7e2dac42ef53a..4672a39cedbdce3 100644 --- a/regression-test/suites/inverted_index_p0/test_tokenize.groovy +++ b/regression-test/suites/inverted_index_p0/test_tokenize.groovy @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +import java.sql.SQLException suite("test_tokenize"){ // prepare test table @@ -98,4 +99,14 @@ suite("test_tokenize"){ qt_tokenize_sql """SELECT TOKENIZE('华夏智胜新税股票A', '"parser"="unicode"');""" qt_tokenize_sql """SELECT TOKENIZE('华夏智胜新税股票A', '"parser"="unicode","stopwords" = "none"');""" + + try { + sql """ SELECT TOKENIZE('华夏智胜新税股票A', '"parser"="eng"'); """ + } catch (SQLException e) { + if (e.message.contains("E-6000")) { + log.info("e message: {}", e.message) + } else { + throw e + } + } } From 449abfac314902067c256f0d1051d9ff513f5f18 Mon Sep 17 00:00:00 2001 From: zzzxl Date: Mon, 25 Nov 2024 17:34:28 +0800 Subject: [PATCH 074/110] [opt](inverted index) Optimize the codes exception handling process (#44205) Problem Summary: 1. Prevent the close exception from happening again --- .../rowset/segment_v2/inverted_index_common.h | 103 ++++++ .../segment_v2/inverted_index_file_writer.cpp | 86 ++--- .../segment_v2/inverted_index_file_writer.h | 8 +- .../segment_v2/inverted_index_writer.cpp | 44 +-- .../common/inverted_index_common_test.cpp | 343 ++++++++++++++++++ .../inverted_index_file_writer_test.cpp | 30 +- ...est_inverted_index_writer_exception.groovy | 89 +++++ 7 files changed, 612 insertions(+), 91 deletions(-) create mode 100644 be/src/olap/rowset/segment_v2/inverted_index_common.h create mode 100644 be/test/olap/rowset/segment_v2/inverted_index/common/inverted_index_common_test.cpp create mode 100644 regression-test/suites/inverted_index_p0/test_inverted_index_writer_exception.groovy diff --git a/be/src/olap/rowset/segment_v2/inverted_index_common.h b/be/src/olap/rowset/segment_v2/inverted_index_common.h new file mode 100644 index 000000000000000..1fdb7df2931de4c --- /dev/null +++ b/be/src/olap/rowset/segment_v2/inverted_index_common.h @@ -0,0 +1,103 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include // IWYU pragma: keep + +#include + +#include "common/logging.h" + +namespace lucene::store { +class Directory; +} // namespace lucene::store + +namespace doris::segment_v2 { + +struct DirectoryDeleter { + void operator()(lucene::store::Directory* ptr) const { _CLDECDELETE(ptr); } +}; + +struct ErrorContext { + std::string err_msg; + std::exception_ptr eptr; +}; + +template +concept HasClose = requires(T t) { + { t->close() }; +}; + +template + requires HasClose +void finally_close(PtrType& resource, ErrorContext& error_context) { + if (resource) { + try { + resource->close(); + } catch (CLuceneError& err) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("Error occurred while closing resource: "); + error_context.err_msg.append(err.what()); + LOG(ERROR) << error_context.err_msg; + } catch (...) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("Error occurred while closing resource"); + LOG(ERROR) << error_context.err_msg; + } + } +} + +#if defined(__clang__) +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wunused-macros" +#endif + +#define FINALLY_CLOSE(resource) \ + { \ + static_assert(sizeof(error_context) > 0, \ + "error_context must be defined before using FINALLY macro!"); \ + finally_close(resource, error_context); \ + } + +// Return ERROR after finally +#define FINALLY(finally_block) \ + { \ + static_assert(sizeof(error_context) > 0, \ + "error_context must be defined before using FINALLY macro!"); \ + finally_block; \ + if (error_context.eptr) { \ + return Status::Error(error_context.err_msg); \ + } \ + } + +// Re-throw the exception after finally +#define FINALLY_EXCEPTION(finally_block) \ + { \ + static_assert(sizeof(error_context) > 0, \ + "error_context must be defined before using FINALLY macro!"); \ + finally_block; \ + if (error_context.eptr) { \ + std::rethrow_exception(error_context.eptr); \ + } \ + } + +#if defined(__clang__) +#pragma clang diagnostic pop +#endif + +} // namespace doris::segment_v2 \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index_file_writer.cpp b/be/src/olap/rowset/segment_v2/inverted_index_file_writer.cpp index 2d50730daffe8a8..bb373be5ee906a5 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_file_writer.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_file_writer.cpp @@ -243,10 +243,9 @@ void InvertedIndexFileWriter::copyFile(const char* fileName, lucene::store::Dire Status InvertedIndexFileWriter::write_v1() { int64_t total_size = 0; - std::string err_msg; - lucene::store::Directory* out_dir = nullptr; - std::exception_ptr eptr; + std::unique_ptr out_dir = nullptr; std::unique_ptr output = nullptr; + ErrorContext error_context; for (const auto& entry : _indices_dirs) { const int64_t index_id = entry.first.first; const auto& index_suffix = entry.first.second; @@ -262,7 +261,7 @@ Status InvertedIndexFileWriter::write_v1() { // Create output stream auto result = create_output_stream_v1(index_id, index_suffix); - out_dir = result.first; + out_dir = std::move(result.first); output = std::move(result.second); size_t start = output->getFilePointer(); @@ -275,23 +274,19 @@ Status InvertedIndexFileWriter::write_v1() { total_size += compound_file_size; add_index_info(index_id, index_suffix, compound_file_size); } catch (CLuceneError& err) { - eptr = std::current_exception(); + error_context.eptr = std::current_exception(); auto index_path = InvertedIndexDescriptor::get_index_file_path_v1( _index_path_prefix, index_id, index_suffix); - err_msg = "CLuceneError occur when write_v1 idx file " + index_path + - " error msg: " + err.what(); - } - - // Close and clean up - finalize_output_dir(out_dir); - if (output) { - output->close(); - } - - if (eptr) { - LOG(ERROR) << err_msg; - return Status::Error(err_msg); + error_context.err_msg.append("CLuceneError occur when write_v1 idx file: "); + error_context.err_msg.append(index_path); + error_context.err_msg.append(", error msg: "); + error_context.err_msg.append(err.what()); + LOG(ERROR) << error_context.err_msg; } + FINALLY({ + FINALLY_CLOSE(output); + FINALLY_CLOSE(out_dir); + }) } _total_file_size = total_size; @@ -299,10 +294,9 @@ Status InvertedIndexFileWriter::write_v1() { } Status InvertedIndexFileWriter::write_v2() { - std::string err_msg; - lucene::store::Directory* out_dir = nullptr; + std::unique_ptr out_dir = nullptr; std::unique_ptr compound_file_output = nullptr; - std::exception_ptr eptr; + ErrorContext error_context; try { // Calculate header length and initialize offset int64_t current_offset = headerLength(); @@ -311,7 +305,7 @@ Status InvertedIndexFileWriter::write_v2() { // Create output stream auto result = create_output_stream_v2(); - out_dir = result.first; + out_dir = std::move(result.first); compound_file_output = std::move(result.second); // Write version and number of indices @@ -326,22 +320,18 @@ Status InvertedIndexFileWriter::write_v2() { _total_file_size = compound_file_output->getFilePointer(); _file_info.set_index_size(_total_file_size); } catch (CLuceneError& err) { - eptr = std::current_exception(); + error_context.eptr = std::current_exception(); auto index_path = InvertedIndexDescriptor::get_index_file_path_v2(_index_path_prefix); - err_msg = "CLuceneError occur when close idx file " + index_path + - " error msg: " + err.what(); - } - - // Close and clean up - finalize_output_dir(out_dir); - if (compound_file_output) { - compound_file_output->close(); - } - - if (eptr) { - LOG(ERROR) << err_msg; - return Status::Error(err_msg); + error_context.err_msg.append("CLuceneError occur when close idx file: "); + error_context.err_msg.append(index_path); + error_context.err_msg.append(", error msg: "); + error_context.err_msg.append(err.what()); + LOG(ERROR) << error_context.err_msg; } + FINALLY({ + FINALLY_CLOSE(compound_file_output); + FINALLY_CLOSE(out_dir); + }) return Status::OK(); } @@ -369,13 +359,6 @@ std::vector InvertedIndexFileWriter::prepare_sorted_files( return sorted_files; } -void InvertedIndexFileWriter::finalize_output_dir(lucene::store::Directory* out_dir) { - if (out_dir != nullptr) { - out_dir->close(); - _CLDECDELETE(out_dir) - } -} - void InvertedIndexFileWriter::add_index_info(int64_t index_id, const std::string& index_suffix, int64_t compound_file_size) { InvertedIndexFileInfo_IndexInfo index_info; @@ -424,7 +407,8 @@ std::pair InvertedIndexFileWriter::calculate_header_length( return {header_length, header_file_count}; } -std::pair> +std::pair, + std::unique_ptr> InvertedIndexFileWriter::create_output_stream_v1(int64_t index_id, const std::string& index_suffix) { io::Path cfs_path(InvertedIndexDescriptor::get_index_file_path_v1(_index_path_prefix, index_id, @@ -434,6 +418,7 @@ InvertedIndexFileWriter::create_output_stream_v1(int64_t index_id, auto* out_dir = DorisFSDirectoryFactory::getDirectory(_fs, idx_path.c_str()); out_dir->set_file_writer_opts(_opts); + std::unique_ptr out_dir_ptr(out_dir); auto* out = out_dir->createOutput(idx_name.c_str()); DBUG_EXECUTE_IF("InvertedIndexFileWriter::write_v1_out_dir_createOutput_nullptr", @@ -443,9 +428,9 @@ InvertedIndexFileWriter::create_output_stream_v1(int64_t index_id, "output is nullptr."; _CLTHROWA(CL_ERR_IO, "Create CompoundDirectory output error"); } - std::unique_ptr output(out); - return {out_dir, std::move(output)}; + + return {std::move(out_dir_ptr), std::move(output)}; } void InvertedIndexFileWriter::write_header_and_data_v1(lucene::store::IndexOutput* output, @@ -483,15 +468,20 @@ void InvertedIndexFileWriter::write_header_and_data_v1(lucene::store::IndexOutpu } } -std::pair> +std::pair, + std::unique_ptr> InvertedIndexFileWriter::create_output_stream_v2() { io::Path index_path {InvertedIndexDescriptor::get_index_file_path_v2(_index_path_prefix)}; + auto* out_dir = DorisFSDirectoryFactory::getDirectory(_fs, index_path.parent_path().c_str()); out_dir->set_file_writer_opts(_opts); + std::unique_ptr out_dir_ptr(out_dir); + DCHECK(_idx_v2_writer != nullptr) << "inverted index file writer v2 is nullptr"; auto compound_file_output = std::unique_ptr( out_dir->createOutputV2(_idx_v2_writer.get())); - return std::make_pair(out_dir, std::move(compound_file_output)); + + return {std::move(out_dir_ptr), std::move(compound_file_output)}; } void InvertedIndexFileWriter::write_version_and_indices_count(lucene::store::IndexOutput* output) { diff --git a/be/src/olap/rowset/segment_v2/inverted_index_file_writer.h b/be/src/olap/rowset/segment_v2/inverted_index_file_writer.h index ddb22975d684670..ba42ffdceb14758 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_file_writer.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_file_writer.h @@ -29,6 +29,7 @@ #include "io/fs/file_system.h" #include "io/fs/file_writer.h" #include "io/fs/local_file_system.h" +#include "olap/rowset/segment_v2/inverted_index_common.h" #include "olap/rowset/segment_v2/inverted_index_desc.h" #include "runtime/exec_env.h" @@ -105,21 +106,22 @@ class InvertedIndexFileWriter { void sort_files(std::vector& file_infos); void copyFile(const char* fileName, lucene::store::Directory* dir, lucene::store::IndexOutput* output, uint8_t* buffer, int64_t bufferLength); - void finalize_output_dir(lucene::store::Directory* out_dir); void add_index_info(int64_t index_id, const std::string& index_suffix, int64_t compound_file_size); int64_t headerLength(); // Helper functions specific to write_v1 std::pair calculate_header_length(const std::vector& sorted_files, lucene::store::Directory* directory); - virtual std::pair> + virtual std::pair, + std::unique_ptr> create_output_stream_v1(int64_t index_id, const std::string& index_suffix); virtual void write_header_and_data_v1(lucene::store::IndexOutput* output, const std::vector& sorted_files, lucene::store::Directory* directory, int64_t header_length, int32_t header_file_count); // Helper functions specific to write_v2 - virtual std::pair> + virtual std::pair, + std::unique_ptr> create_output_stream_v2(); void write_version_and_indices_count(lucene::store::IndexOutput* output); struct FileMetadata { diff --git a/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp b/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp index a4f3ca55dd11c0b..08cebaa07d6670b 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp @@ -51,6 +51,7 @@ #include "olap/rowset/segment_v2/common.h" #include "olap/rowset/segment_v2/inverted_index/analyzer/analyzer.h" #include "olap/rowset/segment_v2/inverted_index/char_filter/char_filter_factory.h" +#include "olap/rowset/segment_v2/inverted_index_common.h" #include "olap/rowset/segment_v2/inverted_index_desc.h" #include "olap/rowset/segment_v2/inverted_index_file_writer.h" #include "olap/rowset/segment_v2/inverted_index_fs_directory.h" @@ -63,11 +64,6 @@ #include "util/slice.h" #include "util/string_util.h" -#define FINALLY_CLOSE_OUTPUT(x) \ - try { \ - if (x != nullptr) x->close(); \ - } catch (...) { \ - } namespace doris::segment_v2 { const int32_t MAX_FIELD_LEN = 0x7FFFFFFFL; const int32_t MERGE_FACTOR = 100000000; @@ -138,13 +134,6 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { } } - void close() { - if (_index_writer) { - _index_writer->close(); - _index_writer.reset(); - } - } - void close_on_error() override { try { DBUG_EXECUTE_IF("InvertedIndexColumnWriter::close_on_error_throw_exception", @@ -618,7 +607,6 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { buf.resize(size); _null_bitmap.write(reinterpret_cast(buf.data()), false); null_bitmap_out->writeBytes(buf.data(), size); - null_bitmap_out->close(); } } @@ -628,6 +616,7 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { std::unique_ptr data_out = nullptr; std::unique_ptr index_out = nullptr; std::unique_ptr meta_out = nullptr; + ErrorContext error_context; try { // write bkd file if constexpr (field_is_numeric_type(field_type)) { @@ -656,16 +645,11 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { << "Inverted index writer create output error occurred: nullptr"; _CLTHROWA(CL_ERR_IO, "Create output error with nullptr"); } - meta_out->close(); - data_out->close(); - index_out->close(); - _dir->close(); } else if constexpr (field_is_slice_type(field_type)) { null_bitmap_out = std::unique_ptr< lucene::store::IndexOutput>(_dir->createOutput( InvertedIndexDescriptor::get_temporary_null_bitmap_file_name())); write_null_bitmap(null_bitmap_out.get()); - close(); DBUG_EXECUTE_IF( "InvertedIndexWriter._throw_clucene_error_in_fulltext_writer_close", { _CLTHROWA(CL_ERR_IO, @@ -673,19 +657,19 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { }); } } catch (CLuceneError& e) { - FINALLY_CLOSE_OUTPUT(null_bitmap_out) - FINALLY_CLOSE_OUTPUT(meta_out) - FINALLY_CLOSE_OUTPUT(data_out) - FINALLY_CLOSE_OUTPUT(index_out) - if constexpr (field_is_numeric_type(field_type)) { - FINALLY_CLOSE_OUTPUT(_dir) - } else if constexpr (field_is_slice_type(field_type)) { - FINALLY_CLOSE_OUTPUT(_index_writer); - } - LOG(WARNING) << "Inverted index writer finish error occurred: " << e.what(); - return Status::Error( - "Inverted index writer finish error occurred:{}", e.what()); + error_context.eptr = std::current_exception(); + error_context.err_msg.append("Inverted index writer finish error occurred: "); + error_context.err_msg.append(e.what()); + LOG(ERROR) << error_context.err_msg; } + FINALLY({ + FINALLY_CLOSE(null_bitmap_out); + FINALLY_CLOSE(meta_out); + FINALLY_CLOSE(data_out); + FINALLY_CLOSE(index_out); + FINALLY_CLOSE(_dir); + FINALLY_CLOSE(_index_writer); + }) return Status::OK(); } diff --git a/be/test/olap/rowset/segment_v2/inverted_index/common/inverted_index_common_test.cpp b/be/test/olap/rowset/segment_v2/inverted_index/common/inverted_index_common_test.cpp new file mode 100644 index 000000000000000..96624260521ab50 --- /dev/null +++ b/be/test/olap/rowset/segment_v2/inverted_index/common/inverted_index_common_test.cpp @@ -0,0 +1,343 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "olap/rowset/segment_v2/inverted_index_common.h" + +#include + +#include "common/status.h" + +namespace doris::segment_v2 { + +class InvertedIndexCommonTest : public testing::Test { +public: + void SetUp() override {} + + void TearDown() override {} + + InvertedIndexCommonTest() = default; + ~InvertedIndexCommonTest() override = default; +}; + +TEST_F(InvertedIndexCommonTest, TestFinallyClose) { + class InvertedIndexBase { + public: + InvertedIndexBase(int32_t& count) : count_(count) {} + + void close() { count_++; } + void clear() { count_++; } + + int32_t& count_; + }; + { + int32_t count = 0; + { + ErrorContext error_context; + auto ptr = std::make_shared(count); + finally_close(ptr, error_context); + } + EXPECT_EQ(count, 1); + } + { + int32_t count = 0; + { + ErrorContext error_context; + auto ptr = std::shared_ptr(new InvertedIndexBase(count), + [](InvertedIndexBase* p) { + if (p) { + p->clear(); + delete p; + p = nullptr; + } + }); + finally_close(ptr, error_context); + } + EXPECT_EQ(count, 2); + } + { + int32_t count = 0; + { + ErrorContext error_context; + auto ptr = std::make_unique(count); + finally_close(ptr, error_context); + } + EXPECT_EQ(count, 1); + } + { + struct Deleter { + void operator()(InvertedIndexBase* p) const { + if (p) { + p->clear(); + delete p; + p = nullptr; + } + } + }; + + int32_t count = 0; + { + ErrorContext error_context; + auto ptr = std::unique_ptr(new InvertedIndexBase(count)); + finally_close(ptr, error_context); + } + EXPECT_EQ(count, 2); + } +} + +TEST_F(InvertedIndexCommonTest, TestTryBlockException) { + class InvertedIndexBase { + public: + void add() { _CLTHROWA(CL_ERR_IO, "test add error"); } + void close() {} + }; + + // return error + { + auto func = []() -> Status { + auto base_ptr = std::make_unique(); + ErrorContext error_context; + try { + base_ptr->add(); + } catch (CLuceneError& e) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("error: "); + error_context.err_msg.append(e.what()); + } + FINALLY({ + EXPECT_TRUE(error_context.eptr); + FINALLY_CLOSE(base_ptr); + }) + return Status::OK(); + }; + auto ret = func(); + EXPECT_EQ(ret.code(), ErrorCode::INVERTED_INDEX_CLUCENE_ERROR); + } + + // throw exception + { + auto func = []() { + auto base_ptr = std::make_unique(); + ErrorContext error_context; + try { + base_ptr->add(); + } catch (CLuceneError& e) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("error: "); + error_context.err_msg.append(e.what()); + } + FINALLY_EXCEPTION({ + EXPECT_TRUE(error_context.eptr); + FINALLY_CLOSE(base_ptr); + }) + }; + bool is_exception = false; + try { + func(); + } catch (CLuceneError& e) { + EXPECT_EQ(e.number(), CL_ERR_IO); + is_exception = true; + } + EXPECT_TRUE(is_exception); + } +} + +TEST_F(InvertedIndexCommonTest, TestFinallyBlockException) { + class InvertedIndexBase { + public: + void add() {} + void close() { _CLTHROWA(CL_ERR_Runtime, "test close error"); } + }; + + // return error + { + auto func = []() -> Status { + auto base_ptr = std::make_unique(); + ErrorContext error_context; + try { + base_ptr->add(); + } catch (CLuceneError& e) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("error: "); + error_context.err_msg.append(e.what()); + } + FINALLY({ + EXPECT_FALSE(error_context.eptr); + FINALLY_CLOSE(base_ptr); + EXPECT_TRUE(error_context.eptr); + }) + return Status::OK(); + }; + auto ret = func(); + EXPECT_EQ(ret.code(), ErrorCode::INVERTED_INDEX_CLUCENE_ERROR); + } + + // throw exception + { + auto func = []() { + auto base_ptr = std::make_unique(); + ErrorContext error_context; + try { + base_ptr->add(); + } catch (CLuceneError& e) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("error: "); + error_context.err_msg.append(e.what()); + } + FINALLY_EXCEPTION({ + EXPECT_FALSE(error_context.eptr); + FINALLY_CLOSE(base_ptr); + EXPECT_TRUE(error_context.eptr); + }) + }; + bool is_exception = false; + try { + func(); + } catch (CLuceneError& e) { + EXPECT_EQ(e.number(), CL_ERR_Runtime); + is_exception = true; + } + EXPECT_TRUE(is_exception); + } +} + +TEST_F(InvertedIndexCommonTest, TestTryAndFinallyBlockException) { + class InvertedIndexBase { + public: + void add() { _CLTHROWA(CL_ERR_IO, "test add error"); } + void close() { _CLTHROWA(CL_ERR_Runtime, "test close error"); } + }; + + // return error + { + auto func = []() -> Status { + auto base_ptr = std::make_unique(); + ErrorContext error_context; + try { + base_ptr->add(); + } catch (CLuceneError& e) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("error: "); + error_context.err_msg.append(e.what()); + } + FINALLY({ + EXPECT_TRUE(error_context.eptr); + FINALLY_CLOSE(base_ptr); + EXPECT_TRUE(error_context.eptr); + }) + return Status::OK(); + }; + auto ret = func(); + EXPECT_EQ(ret.code(), ErrorCode::INVERTED_INDEX_CLUCENE_ERROR); + } + + // throw exception + { + auto func = []() { + auto base_ptr = std::make_unique(); + ErrorContext error_context; + try { + base_ptr->add(); + } catch (CLuceneError& e) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("error: "); + error_context.err_msg.append(e.what()); + } + FINALLY_EXCEPTION({ + EXPECT_TRUE(error_context.eptr); + FINALLY_CLOSE(base_ptr); + EXPECT_TRUE(error_context.eptr); + }) + }; + bool is_exception = false; + try { + func(); + } catch (CLuceneError& e) { + EXPECT_EQ(e.number(), CL_ERR_Runtime); + is_exception = true; + } + EXPECT_TRUE(is_exception); + } +} + +TEST_F(InvertedIndexCommonTest, TestRawPointerException) { + class InvertedIndexBase { + public: + void add() { _CLTHROWA(CL_ERR_IO, "test add error"); } + void close() { _CLTHROWA(CL_ERR_Runtime, "test close error"); } + }; + + // return error + { + auto func = []() -> Status { + auto* base_ptr = new InvertedIndexBase(); + ErrorContext error_context; + try { + base_ptr->add(); + } catch (CLuceneError& e) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("error: "); + error_context.err_msg.append(e.what()); + } + FINALLY({ + EXPECT_TRUE(error_context.eptr); + FINALLY_CLOSE(base_ptr); + if (base_ptr) { + delete base_ptr; + base_ptr = nullptr; + } + EXPECT_TRUE(error_context.eptr); + }) + return Status::OK(); + }; + auto ret = func(); + EXPECT_EQ(ret.code(), ErrorCode::INVERTED_INDEX_CLUCENE_ERROR); + } + + // throw exception + { + auto func = []() { + auto* base_ptr = new InvertedIndexBase(); + ErrorContext error_context; + try { + base_ptr->add(); + } catch (CLuceneError& e) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("error: "); + error_context.err_msg.append(e.what()); + } + FINALLY_EXCEPTION({ + EXPECT_TRUE(error_context.eptr); + FINALLY_CLOSE(base_ptr); + if (base_ptr) { + delete base_ptr; + base_ptr = nullptr; + } + EXPECT_TRUE(error_context.eptr); + }) + }; + bool is_exception = false; + try { + func(); + } catch (CLuceneError& e) { + EXPECT_EQ(e.number(), CL_ERR_Runtime); + is_exception = true; + } + EXPECT_TRUE(is_exception); + } +} + +} // namespace doris::segment_v2 \ No newline at end of file diff --git a/be/test/olap/rowset/segment_v2/inverted_index_file_writer_test.cpp b/be/test/olap/rowset/segment_v2/inverted_index_file_writer_test.cpp index 460b99a0ce785e3..b454080434a008c 100644 --- a/be/test/olap/rowset/segment_v2/inverted_index_file_writer_test.cpp +++ b/be/test/olap/rowset/segment_v2/inverted_index_file_writer_test.cpp @@ -521,7 +521,8 @@ class InvertedIndexFileWriterMockCreateOutputStreamV2 : public InvertedIndexFile : InvertedIndexFileWriter(fs, index_path_prefix, rowset_id, segment_id, storage_format) {} - MOCK_METHOD((std::pair>), + MOCK_METHOD((std::pair, + std::unique_ptr>), create_output_stream_v2, (), (override)); }; @@ -535,7 +536,8 @@ class InvertedIndexFileWriterMockCreateOutputStreamV1 : public InvertedIndexFile : InvertedIndexFileWriter(fs, index_path_prefix, rowset_id, segment_id, storage_format) {} - MOCK_METHOD((std::pair>), + MOCK_METHOD((std::pair, + std::unique_ptr>), create_output_stream_v1, (int64_t index_id, const std::string& index_suffix), (override)); }; @@ -571,6 +573,7 @@ TEST_F(InvertedIndexFileWriterTest, WriteV1OutputTest) { std::string idx_name = cfs_path.filename(); auto* out_dir = DorisFSDirectoryFactory::getDirectory(_fs, idx_path.c_str()); + std::unique_ptr out_dir_ptr(out_dir); auto* mock_output_v1 = new MockFSIndexOutputV1(); EXPECT_CALL(*mock_output_v1, flushBuffer(::testing::_, ::testing::_)) .WillOnce(::testing::Throw(CLuceneError(CL_ERR_IO, "Simulated exception", false))); @@ -579,9 +582,10 @@ TEST_F(InvertedIndexFileWriterTest, WriteV1OutputTest) { EXPECT_CALL(writer_mock, create_output_stream_v1(index_id, index_suffix)) .WillOnce(::testing::Invoke( - [&]() -> std::pair std::pair, std::unique_ptr> { - return std::make_pair(out_dir, std::move(compound_file_output)); + return std::make_pair(std::move(out_dir_ptr), + std::move(compound_file_output)); })); auto index_meta = create_mock_tablet_index(index_id, index_suffix); @@ -611,6 +615,7 @@ TEST_F(InvertedIndexFileWriterTest, WriteV2OutputTest) { InvertedIndexFileWriterMockCreateOutputStreamV2 writer_mock( _fs, _index_path_prefix, _rowset_id, _seg_id, InvertedIndexStorageFormatPB::V2); auto* out_dir = DorisFSDirectoryFactory::getDirectory(_fs, index_path.c_str()); + std::unique_ptr out_dir_ptr(out_dir); auto* mock_output_v2 = new MockFSIndexOutputV2(); EXPECT_CALL(*mock_output_v2, flushBuffer(::testing::_, ::testing::_)) .WillOnce(::testing::Throw(CLuceneError(CL_ERR_IO, "Simulated exception", false))); @@ -619,9 +624,10 @@ TEST_F(InvertedIndexFileWriterTest, WriteV2OutputTest) { EXPECT_CALL(writer_mock, create_output_stream_v2()) .WillOnce(::testing::Invoke( - [&]() -> std::pair std::pair, std::unique_ptr> { - return std::make_pair(out_dir, std::move(compound_file_output)); + return std::make_pair(std::move(out_dir_ptr), + std::move(compound_file_output)); })); int64_t index_id = 1; @@ -665,6 +671,7 @@ TEST_F(InvertedIndexFileWriterTest, WriteV2OutputCloseErrorTest) { InvertedIndexFileWriterMockCreateOutputStreamV2 writer_mock( _fs, _index_path_prefix, _rowset_id, _seg_id, InvertedIndexStorageFormatPB::V2); auto* out_dir = DorisFSDirectoryFactory::getDirectory(_fs, index_path.c_str()); + std::unique_ptr out_dir_ptr(out_dir); auto* mock_output_v2 = new MockFSIndexOutputCloseV2(); EXPECT_CALL(*mock_output_v2, close()).WillOnce(::testing::Invoke([&]() { mock_output_v2->base_close(); @@ -675,9 +682,10 @@ TEST_F(InvertedIndexFileWriterTest, WriteV2OutputCloseErrorTest) { EXPECT_CALL(writer_mock, create_output_stream_v2()) .WillOnce(::testing::Invoke( - [&]() -> std::pair std::pair, std::unique_ptr> { - return std::make_pair(out_dir, std::move(compound_file_output)); + return std::make_pair(std::move(out_dir_ptr), + std::move(compound_file_output)); })); int64_t index_id = 1; @@ -710,6 +718,7 @@ TEST_F(InvertedIndexFileWriterTest, WriteV1OutputCloseErrorTest) { std::string idx_name = cfs_path.filename(); auto* out_dir = DorisFSDirectoryFactory::getDirectory(_fs, idx_path.c_str()); + std::unique_ptr out_dir_ptr(out_dir); auto* mock_output_v1 = new MockFSIndexOutputCloseV1(); EXPECT_CALL(*mock_output_v1, close()).WillOnce(::testing::Invoke([&]() { mock_output_v1->base_close(); @@ -720,9 +729,10 @@ TEST_F(InvertedIndexFileWriterTest, WriteV1OutputCloseErrorTest) { EXPECT_CALL(writer_mock, create_output_stream_v1(index_id, index_suffix)) .WillOnce(::testing::Invoke( - [&]() -> std::pair std::pair, std::unique_ptr> { - return std::make_pair(out_dir, std::move(compound_file_output)); + return std::make_pair(std::move(out_dir_ptr), + std::move(compound_file_output)); })); auto index_meta = create_mock_tablet_index(index_id, index_suffix); diff --git a/regression-test/suites/inverted_index_p0/test_inverted_index_writer_exception.groovy b/regression-test/suites/inverted_index_p0/test_inverted_index_writer_exception.groovy new file mode 100644 index 000000000000000..ced1c8d74aecfc8 --- /dev/null +++ b/regression-test/suites/inverted_index_p0/test_inverted_index_writer_exception.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. + +import java.sql.SQLException + +suite("test_inverted_index_writer_exception", "nonConcurrent"){ + def indexTbName1 = "test_inverted_index_writer_exception" + + sql "DROP TABLE IF EXISTS ${indexTbName1}" + + sql """ + CREATE TABLE ${indexTbName1} ( + `@timestamp` int(11) NULL COMMENT "", + `clientip` varchar(20) NULL COMMENT "", + `request` text NULL COMMENT "", + `status` int(11) NULL COMMENT "", + `size` int(11) NULL COMMENT "", + INDEX request_idx (`request`) USING INVERTED PROPERTIES("parser" = "english") COMMENT '' + ) ENGINE=OLAP + DUPLICATE KEY(`@timestamp`) + COMMENT "OLAP" + DISTRIBUTED BY RANDOM BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + try { + GetDebugPoint().enableDebugPointForAllBEs("InvertedIndexWriter._throw_clucene_error_in_fulltext_writer_close") + try { + sql """ INSERT INTO ${indexTbName1} VALUES (1, "40.135.0.0", "GET /images/hm_bg.jpg HTTP/1.0", 1, 2); """ + } catch (SQLException e) { + if (e.message.contains("E-6002")) { + log.info("Test passed 1: Encountered expected exception [E-6002].") + } else { + throw e + } + } + } finally { + GetDebugPoint().disableDebugPointForAllBEs("InvertedIndexWriter._throw_clucene_error_in_fulltext_writer_close") + } + + try { + GetDebugPoint().enableDebugPointForAllBEs("DorisFSDirectory::close_close_with_error") + try { + sql """ INSERT INTO ${indexTbName1} VALUES (2, "40.135.0.0", "GET /images/hm_bg.jpg HTTP/1.0", 1, 2); """ + } catch (SQLException e) { + if (e.message.contains("E-6002")) { + log.info("Test passed 2: Encountered expected exception [E-6002].") + } else { + throw e + } + } + } finally { + GetDebugPoint().disableDebugPointForAllBEs("DorisFSDirectory::close_close_with_error") + } + + try { + GetDebugPoint().enableDebugPointForAllBEs("InvertedIndexWriter._throw_clucene_error_in_fulltext_writer_close") + GetDebugPoint().enableDebugPointForAllBEs("DorisFSDirectory::close_close_with_error") + + try { + sql """ INSERT INTO ${indexTbName1} VALUES (3, "40.135.0.0", "GET /images/hm_bg.jpg HTTP/1.0", 1, 2); """ + } catch (SQLException e) { + if (e.message.contains("E-6002")) { + log.info("Test passed 3: Encountered expected exception [E-6002].") + } else { + throw e + } + } + } finally { + GetDebugPoint().disableDebugPointForAllBEs("InvertedIndexWriter._throw_clucene_error_in_fulltext_writer_close") + GetDebugPoint().disableDebugPointForAllBEs("DorisFSDirectory::close_close_with_error") + } +} \ No newline at end of file From 3412da431eca7f033c7c8a4f9717dec757605c68 Mon Sep 17 00:00:00 2001 From: starocean999 Date: Mon, 25 Nov 2024 17:44:59 +0800 Subject: [PATCH 075/110] [fix](nereids)lag and lead function's nullability is wrong (#44050) ### What problem does this PR solve? lag and lead function's nullability should depends on either first or third( if it has) param's nullability. --- .../expressions/functions/window/Lag.java | 3 +- .../expressions/functions/window/Lead.java | 3 +- .../test_lead_lag_nullable.groovy | 80 +++++++++++++++++++ 3 files changed, 82 insertions(+), 4 deletions(-) create mode 100644 regression-test/suites/nereids_p0/sql_functions/window_functions/test_lead_lag_nullable.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Lag.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Lag.java index 42b41a1355661a9..7f7c9511e8167a0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Lag.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Lag.java @@ -22,7 +22,6 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.Literal; -import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.trees.expressions.shape.TernaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BigIntType; @@ -72,7 +71,7 @@ public Expression getDefaultValue() { @Override public boolean nullable() { - if (children.size() == 3 && child(2) instanceof NullLiteral) { + if (children.size() == 3 && child(2).nullable()) { return true; } return child(0).nullable(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Lead.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Lead.java index 004b5195c0cdaa8..ec6a4b7b85c2003 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Lead.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Lead.java @@ -22,7 +22,6 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.Literal; -import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.trees.expressions.shape.TernaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BigIntType; @@ -74,7 +73,7 @@ public Expression getDefaultValue() { @Override public boolean nullable() { - if (children.size() == 3 && child(2) instanceof NullLiteral) { + if (children.size() == 3 && child(2).nullable()) { return true; } return child(0).nullable(); diff --git a/regression-test/suites/nereids_p0/sql_functions/window_functions/test_lead_lag_nullable.groovy b/regression-test/suites/nereids_p0/sql_functions/window_functions/test_lead_lag_nullable.groovy new file mode 100644 index 000000000000000..93f863eb1b19f4b --- /dev/null +++ b/regression-test/suites/nereids_p0/sql_functions/window_functions/test_lead_lag_nullable.groovy @@ -0,0 +1,80 @@ +// 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_lead_lag_nullable") { + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + sql """DROP TABLE IF EXISTS `test_lead_lag_ttt`; """ + sql """ + CREATE TABLE `test_lead_lag_ttt` ( + `wtid` varchar(72) NOT NULL , + `rectime` datetime NOT NULL , + `id` bigint NOT NULL , + `if_delete` tinyint NOT NULL DEFAULT "0" , + `endtime` datetime NULL , + `std_state_code` int NULL , + `stop_reason_code` int NULL , + `remark` varchar(100) NULL, + `if_raw` tinyint NULL DEFAULT "1" , + `create_time` datetime NULL , + `update_time` datetime NULL , + `uuid` varchar(50) NOT NULL + ) ENGINE=OLAP + UNIQUE KEY(`wtid`, `rectime`, `id`, `if_delete`) + DISTRIBUTED BY HASH(`wtid`) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1"); + """ + sql """INSERT INTO test_lead_lag_ttt + (wtid, rectime, id, if_delete, endtime, std_state_code, stop_reason_code, remark, if_raw, create_time, update_time, uuid) + VALUES + ('device001', '2024-11-15 08:00:00', 1, 0, NULL, 100, 5, '备注信息1', 1, '2024-11-15 08:10:00', '2024-11-15 08:20:00', 'uuid-0001'), + ('device001', '2024-11-15 09:00:00', 2, 0, NULL, 101, 10, '备注信息2', 1, '2024-11-15 09:10:00', '2024-11-15 09:20:00', 'uuid-0002'), + ('device002', '2024-11-15 10:00:00', 3, 0, NULL, 102, 20, '备注信息3', 1, '2024-11-15 10:10:00', '2024-11-15 10:20:00', 'uuid-0003'), + ('device003', '2024-11-15 11:00:00', 4, 0, NULL, 103, 15, '备注信息4', 1, '2024-11-15 11:10:00', '2024-11-15 11:20:00', 'uuid-0004'); """ + + sql """INSERT + INTO + test_lead_lag_ttt (id, + uuid, + wtid, + create_time, + rectime, + if_delete, + std_state_code, + stop_reason_code, + endtime) + SELECT + id, + uuid, + wtid, + LAG(rectime, 1, NULL) OVER (PARTITION BY wtid + ORDER BY + rectime) AS create_time, + rectime, + if_delete, + std_state_code, + stop_reason_code, + LEAD(rectime, 1, NULL) OVER (PARTITION BY wtid + ORDER BY + rectime) AS endtime + FROM + test_lead_lag_ttt + WHERE + endtime is null; """ +} + From 7498239558158f971e1e1cf7513121d83858f63c Mon Sep 17 00:00:00 2001 From: huanghaibin Date: Mon, 25 Nov 2024 17:46:53 +0800 Subject: [PATCH 076/110] [improve](cloud-mow) Optimize log output when missing delete bitmap cache (#44446) Problem Summary: When there is new rowset genarate, query node will not find the delete bitmap of this new rowset in cache, because cache only exists on the write node, so need to sync delete bitmap from ms, this pr reduce the reduce the unnecessary log output about not found cache. --- be/src/cloud/cloud_meta_mgr.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/be/src/cloud/cloud_meta_mgr.cpp b/be/src/cloud/cloud_meta_mgr.cpp index 05341d0d4bab824..5142d897475e03b 100644 --- a/be/src/cloud/cloud_meta_mgr.cpp +++ b/be/src/cloud/cloud_meta_mgr.cpp @@ -610,8 +610,9 @@ bool CloudMetaMgr::sync_tablet_delete_bitmap_by_cache(CloudTablet* tablet, int64 engine.txn_delete_bitmap_cache().remove_unused_tablet_txn_info(txn_id, tablet->tablet_id()); } else { - LOG(WARNING) << "failed to get tablet txn info. tablet_id=" << tablet->tablet_id() - << ", txn_id=" << txn_id << ", status=" << status; + LOG_EVERY_N(INFO, 20) + << "delete bitmap not found in cache, will sync rowset to get. tablet_id= " + << tablet->tablet_id() << ", txn_id=" << txn_id << ", status=" << status; return false; } } @@ -630,8 +631,6 @@ Status CloudMetaMgr::sync_tablet_delete_bitmap(CloudTablet* tablet, int64_t old_ sync_tablet_delete_bitmap_by_cache(tablet, old_max_version, rs_metas, delete_bitmap)) { return Status::OK(); } else { - LOG(WARNING) << "failed to sync delete bitmap by txn info. tablet_id=" - << tablet->tablet_id(); DeleteBitmapPtr new_delete_bitmap = std::make_shared(tablet->tablet_id()); *delete_bitmap = *new_delete_bitmap; } From 36ae71c8a6568479de1c4449dfbf651ec08ad0d2 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Mon, 25 Nov 2024 19:35:12 +0800 Subject: [PATCH 077/110] [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 --- .../doris/datasource/hive/HMSExternalTable.java | 9 +++++++-- .../rules/OneListPartitionEvaluator.java | 14 +++++++------- .../expression/rules/OnePartitionEvaluator.java | 4 ++-- .../rules/OneRangePartitionEvaluator.java | 14 +++++++------- .../rules/expression/rules/PartitionPruner.java | 16 ++++++++-------- .../rules/UnknownPartitionEvaluator.java | 12 ++++++------ .../rules/rewrite/PruneFileScanPartition.java | 14 +++++++------- .../trees/plans/logical/LogicalFileScan.java | 6 +++--- 8 files changed, 47 insertions(+), 42 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 98984467d75b5c6..689b5a9b59a4d44 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 @@ -308,8 +308,13 @@ public SelectedPartitions getAllPartitions() { HiveMetaStoreCache.HivePartitionValues hivePartitionValues = cache.getPartitionValues( this.getDbName(), this.getName(), partitionColumnTypes); Map idToPartitionItem = hivePartitionValues.getIdToPartitionItem(); - - return new SelectedPartitions(idToPartitionItem.size(), idToPartitionItem, false); + // transfer id to name + BiMap idToName = hivePartitionValues.getPartitionNameToIdMap().inverse(); + Map nameToPartitionItem = Maps.newHashMapWithExpectedSize(idToPartitionItem.size()); + for (Entry entry : idToPartitionItem.entrySet()) { + nameToPartitionItem.put(idToName.get(entry.getKey()), entry.getValue()); + } + return new SelectedPartitions(idToPartitionItem.size(), nameToPartitionItem, false); } public boolean isHiveTransactionalTable() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneListPartitionEvaluator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneListPartitionEvaluator.java index b9bdf520e3d6d4d..ecf8a26724113fe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneListPartitionEvaluator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneListPartitionEvaluator.java @@ -35,16 +35,16 @@ import java.util.stream.IntStream; /** OneListPartitionInputs */ -public class OneListPartitionEvaluator - extends DefaultExpressionRewriter> implements OnePartitionEvaluator { - private final long partitionId; +public class OneListPartitionEvaluator + extends DefaultExpressionRewriter> implements OnePartitionEvaluator { + private final K partitionIdent; private final List partitionSlots; private final ListPartitionItem partitionItem; private final ExpressionRewriteContext expressionRewriteContext; - public OneListPartitionEvaluator(long partitionId, List partitionSlots, + public OneListPartitionEvaluator(K partitionIdent, List partitionSlots, ListPartitionItem partitionItem, CascadesContext cascadesContext) { - this.partitionId = partitionId; + this.partitionIdent = partitionIdent; this.partitionSlots = Objects.requireNonNull(partitionSlots, "partitionSlots cannot be null"); this.partitionItem = Objects.requireNonNull(partitionItem, "partitionItem cannot be null"); this.expressionRewriteContext = new ExpressionRewriteContext( @@ -52,8 +52,8 @@ public OneListPartitionEvaluator(long partitionId, List partitionSlots, } @Override - public long getPartitionId() { - return partitionId; + public K getPartitionIdent() { + return partitionIdent; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OnePartitionEvaluator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OnePartitionEvaluator.java index c51252b44a624d5..8810a04750f792f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OnePartitionEvaluator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OnePartitionEvaluator.java @@ -25,8 +25,8 @@ import java.util.Map; /** the evaluator of the partition which represent one partition */ -public interface OnePartitionEvaluator { - long getPartitionId(); +public interface OnePartitionEvaluator { + K getPartitionIdent(); /** * return a slot to expression mapping to replace the input. diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java index 84a037171f32c59..1fb8954ab165475 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java @@ -80,10 +80,10 @@ * * you can see the process steps in the comment of PartitionSlotInput.columnRanges */ -public class OneRangePartitionEvaluator +public class OneRangePartitionEvaluator extends ExpressionVisitor - implements OnePartitionEvaluator { - private final long partitionId; + implements OnePartitionEvaluator { + private final K partitionIdent; private final List partitionSlots; private final RangePartitionItem partitionItem; private final ExpressionRewriteContext expressionRewriteContext; @@ -95,9 +95,9 @@ public class OneRangePartitionEvaluator private final Map slotToType; /** OneRangePartitionEvaluator */ - public OneRangePartitionEvaluator(long partitionId, List partitionSlots, + public OneRangePartitionEvaluator(K partitionIdent, List partitionSlots, RangePartitionItem partitionItem, CascadesContext cascadesContext, int expandThreshold) { - this.partitionId = partitionId; + this.partitionIdent = partitionIdent; this.partitionSlots = Objects.requireNonNull(partitionSlots, "partitionSlots cannot be null"); this.partitionItem = Objects.requireNonNull(partitionItem, "partitionItem cannot be null"); this.expressionRewriteContext = new ExpressionRewriteContext( @@ -155,8 +155,8 @@ public OneRangePartitionEvaluator(long partitionId, List partitionSlots, } @Override - public long getPartitionId() { - return partitionId; + public K getPartitionIdent() { + return partitionIdent; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java index efe12f38cd74e4d..fac1a7f82d2cfb8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java @@ -102,21 +102,21 @@ public Expression visitComparisonPredicate(ComparisonPredicate cp, Void context) } /** prune */ - public List prune() { - Builder scanPartitionIds = ImmutableList.builder(); + public List prune() { + Builder scanPartitionIdents = ImmutableList.builder(); for (OnePartitionEvaluator partition : partitions) { if (!canBePrunedOut(partition)) { - scanPartitionIds.add(partition.getPartitionId()); + scanPartitionIdents.add((K) partition.getPartitionIdent()); } } - return scanPartitionIds.build(); + return scanPartitionIdents.build(); } /** * prune partition with `idToPartitions` as parameter. */ - public static List prune(List partitionSlots, Expression partitionPredicate, - Map idToPartitions, CascadesContext cascadesContext, + public static List prune(List partitionSlots, Expression partitionPredicate, + Map idToPartitions, CascadesContext cascadesContext, PartitionTableType partitionTableType) { partitionPredicate = PartitionPruneExpressionExtractor.extract( partitionPredicate, ImmutableSet.copyOf(partitionSlots), cascadesContext); @@ -135,7 +135,7 @@ public static List prune(List partitionSlots, Expression partitionPr } List evaluators = Lists.newArrayListWithCapacity(idToPartitions.size()); - for (Entry kv : idToPartitions.entrySet()) { + for (Entry kv : idToPartitions.entrySet()) { evaluators.add(toPartitionEvaluator( kv.getKey(), kv.getValue(), partitionSlots, cascadesContext, expandThreshold)); } @@ -147,7 +147,7 @@ public static List prune(List partitionSlots, Expression partitionPr /** * convert partition item to partition evaluator */ - public static final OnePartitionEvaluator toPartitionEvaluator(long id, PartitionItem partitionItem, + public static final OnePartitionEvaluator toPartitionEvaluator(K id, PartitionItem partitionItem, List partitionSlots, CascadesContext cascadesContext, int expandThreshold) { if (partitionItem instanceof ListPartitionItem) { return new OneListPartitionEvaluator( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/UnknownPartitionEvaluator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/UnknownPartitionEvaluator.java index ae313ca09de2697..394182a1311484c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/UnknownPartitionEvaluator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/UnknownPartitionEvaluator.java @@ -28,18 +28,18 @@ import java.util.Map; /** UnknownPartitionEvaluator */ -public class UnknownPartitionEvaluator implements OnePartitionEvaluator { - private final long partitionId; +public class UnknownPartitionEvaluator implements OnePartitionEvaluator { + private final K partitionIdent; private final PartitionItem partitionItem; - public UnknownPartitionEvaluator(long partitionId, PartitionItem partitionItem) { - this.partitionId = partitionId; + public UnknownPartitionEvaluator(K partitionId, PartitionItem partitionItem) { + this.partitionIdent = partitionId; this.partitionItem = partitionItem; } @Override - public long getPartitionId() { - return partitionId; + public K getPartitionIdent() { + return partitionIdent; } @Override 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 2de4efab2ff6ed9..9bec65708222408 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 @@ -76,7 +76,7 @@ public Rule build() { private SelectedPartitions pruneHivePartitions(HMSExternalTable hiveTbl, LogicalFilter filter, LogicalFileScan scan, CascadesContext ctx) { - Map selectedPartitionItems = Maps.newHashMap(); + Map selectedPartitionItems = Maps.newHashMap(); if (CollectionUtils.isEmpty(hiveTbl.getPartitionColumns())) { // non partitioned table, return NOT_PRUNED. // non partition table will be handled in HiveScanNode. @@ -91,13 +91,13 @@ private SelectedPartitions pruneHivePartitions(HMSExternalTable hiveTbl, .map(column -> scanOutput.get(column.getName().toLowerCase())) .collect(Collectors.toList()); - Map idToPartitionItem = scan.getSelectedPartitions().selectedPartitions; - List prunedPartitions = new ArrayList<>(PartitionPruner.prune( - partitionSlots, filter.getPredicate(), idToPartitionItem, ctx, PartitionTableType.HIVE)); + Map nameToPartitionItem = scan.getSelectedPartitions().selectedPartitions; + List prunedPartitions = new ArrayList<>(PartitionPruner.prune( + partitionSlots, filter.getPredicate(), nameToPartitionItem, ctx, PartitionTableType.HIVE)); - for (Long id : prunedPartitions) { - selectedPartitionItems.put(id, idToPartitionItem.get(id)); + for (String name : prunedPartitions) { + selectedPartitionItems.put(name, nameToPartitionItem.get(name)); } - return new SelectedPartitions(idToPartitionItem.size(), selectedPartitionItems, true); + return new SelectedPartitions(nameToPartitionItem.size(), selectedPartitionItems, true); } } 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 0a2c69b68c1d33a..ab4ef8efa3c5dfd 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 @@ -147,9 +147,9 @@ public static class SelectedPartitions { */ public final long totalPartitionNum; /** - * partition id -> partition item + * partition name -> partition item */ - public final Map selectedPartitions; + public final Map selectedPartitions; /** * true means the result is after partition pruning * false means the partition pruning is not processed. @@ -159,7 +159,7 @@ public static class SelectedPartitions { /** * Constructor for SelectedPartitions. */ - public SelectedPartitions(long totalPartitionNum, Map selectedPartitions, + public SelectedPartitions(long totalPartitionNum, Map selectedPartitions, boolean isPruned) { this.totalPartitionNum = totalPartitionNum; this.selectedPartitions = ImmutableMap.copyOf(Objects.requireNonNull(selectedPartitions, From b978788833dbd3b8507bd48f15dccf05e11aa1ad Mon Sep 17 00:00:00 2001 From: seawinde Date: Mon, 25 Nov 2024 19:38:53 +0800 Subject: [PATCH 078/110] [opt](mtmv) Support any_value aggregate function rollup rewrite when exsits aggregate materialized view (#44412) ### What problem does this PR solve? Support any_value rewrite when aggregate materialized view Such as mv def sql is as following select o_orderdate, any_value(l_partkey), o_orderstatus, l_suppkey, any_value(l_suppkey), any_value(o_orderstatus) from lineitem_2 left join orders_2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate where l_partkey = 3 group by o_orderstatus, o_orderdate, l_partkey, l_suppkey; the query like following can be rewritten scuucessfully by mv above select any_value(l_partkey), any_value(l_suppkey), any_value(o_orderstatus) from lineitem_2 left join orders_2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate where l_partkey = 3 and o_orderstatus = 'o' group by o_orderdate, l_suppkey; ### Release note Support any_value aggregate function rollup rewrite when exsits aggregate materialized view --- ...AbstractMaterializedViewAggregateRule.java | 83 ++-- ...lizedViewAggregateOnNoneAggregateRule.java | 2 +- .../mv/rollup/AggFunctionRollUpHandler.java | 6 +- .../ContainDistinctFunctionRollupHandler.java | 1 + .../mv/rollup/DirectRollupHandler.java | 4 +- .../mv/rollup/MappingRollupHandler.java | 28 +- .../expressions/functions/agg/AnyValue.java | 13 +- .../mv/agg_with_roll_up/any_value_roll_up.out | 73 +++ .../agg_with_roll_up/any_value_roll_up.groovy | 417 ++++++++++++++++++ 9 files changed, 561 insertions(+), 66 deletions(-) create mode 100644 regression-test/data/nereids_rules_p0/mv/agg_with_roll_up/any_value_roll_up.out create mode 100644 regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/any_value_roll_up.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java index 1a66eda2ad3eed6..743003dca6bb81e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java @@ -50,7 +50,6 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.algebra.Repeat; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; -import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalRepeat; @@ -155,7 +154,7 @@ protected Plan rewriteQueryByView(MatchMode matchMode, materializationContext.getShuttledExprToScanExprMapping(), viewToQuerySlotMapping)); } - return doRewriteQueryByView(queryStructInfo, + return aggregateRewriteByView(queryStructInfo, viewToQuerySlotMapping, queryTopPlanAndAggPair, tempRewritedPlan, @@ -167,7 +166,7 @@ protected Plan rewriteQueryByView(MatchMode matchMode, /** * Aggregate function and group by expression rewrite impl */ - protected LogicalAggregate doRewriteQueryByView( + protected LogicalAggregate aggregateRewriteByView( StructInfo queryStructInfo, SlotMapping viewToQuerySlotMapping, Pair> queryTopPlanAndAggPair, @@ -426,12 +425,11 @@ private boolean isGroupByEquals(Pair> queryTopPlanA LogicalAggregate queryAggregate = queryTopPlanAndAggPair.value(); LogicalAggregate viewAggregate = viewTopPlanAndAggPair.value(); - Set queryGroupShuttledExpression = new HashSet<>(ExpressionUtils.shuttleExpressionWithLineage( + Set queryGroupByShuttledExpression = new HashSet<>(ExpressionUtils.shuttleExpressionWithLineage( queryAggregate.getGroupByExpressions(), queryTopPlan, queryStructInfo.getTableBitSet())); // try to eliminate group by dimension by function dependency if group by expression is not in query Map viewShuttledExpressionQueryBasedToGroupByExpressionMap = new HashMap<>(); - Map groupByExpressionToViewShuttledExpressionQueryBasedMap = new HashMap<>(); List viewGroupByExpressions = viewAggregate.getGroupByExpressions(); List viewGroupByShuttledExpressions = ExpressionUtils.shuttleExpressionWithLineage( viewGroupByExpressions, viewTopPlan, viewStructInfo.getTableBitSet()); @@ -443,43 +441,40 @@ private boolean isGroupByEquals(Pair> queryTopPlanA viewToQuerySlotMapping.toSlotReferenceMap()); viewShuttledExpressionQueryBasedToGroupByExpressionMap.put(viewGroupExpressionQueryBased, viewExpression); - groupByExpressionToViewShuttledExpressionQueryBasedMap.put(viewExpression, - viewGroupExpressionQueryBased - ); } - if (queryGroupShuttledExpression.equals(viewShuttledExpressionQueryBasedToGroupByExpressionMap.keySet())) { + if (queryGroupByShuttledExpression.equals(viewShuttledExpressionQueryBasedToGroupByExpressionMap.keySet())) { // return true, if equals directly return true; } - - boolean isGroupByEquals = false; - // check is equals by group by eliminate - isGroupByEquals |= isGroupByEqualsAfterGroupByEliminate(queryGroupShuttledExpression, - viewShuttledExpressionQueryBasedToGroupByExpressionMap, - groupByExpressionToViewShuttledExpressionQueryBasedMap, - viewAggregate, - cascadesContext); - // check is equals by equal filter eliminate - Optional> filterOptional = tempRewrittenPlan.collectFirst(LogicalFilter.class::isInstance); - if (!filterOptional.isPresent()) { - return isGroupByEquals; - } - isGroupByEquals |= isGroupByEqualsAfterEqualFilterEliminate( + // Check is equals by equal filter eliminate + return isGroupByEqualsByFunctionDependency( (LogicalPlan) tempRewrittenPlan, - queryGroupShuttledExpression, + queryGroupByShuttledExpression, viewShuttledExpressionQueryBasedToGroupByExpressionMap, materializationContext); - return isGroupByEquals; } /** - * Check group by is equals by equal filter eliminate - * For example query is select a, b, c from t1 where a = 1 and d = 'xx' group by a, b, c; - * mv is select a, b, c, d from t1 group by a, b, c, d; - * the group by expression between query and view is equals after equal filter eliminate - * should not aggregate roll up + * Check group by is equals by uniform function dependency + * For example query is: + * select + * a, b, c from t1 + * where a = 1 and d = 'xx' + * group by a, b, c; + * mv is : + * select a, b, c, d + * from t1 + * group by a, b, c, d; + * After group by key eliminate, the query group by is b, c + * but mv is group by a, b, c, d, the group by a and d of mv is more dimensions than the query + * But in tempRewrittenPlan is as following: + * select * + * from mv + * where a = 1 and d = 'xx' + * We can get group by a and d is uniform by function dependency info, + * so the group by expression between query and view is equals, should not aggregate roll up * */ - private static boolean isGroupByEqualsAfterEqualFilterEliminate( + private static boolean isGroupByEqualsByFunctionDependency( LogicalPlan tempRewrittenPlan, Set queryGroupShuttledExpression, Map viewShuttledExprQueryBasedToViewGroupByExprMap, @@ -541,42 +536,42 @@ private static boolean isGroupByEqualsAfterEqualFilterEliminate( * viewAggregate * 4. check the viewAggregate group by expression is equals queryAggregate expression or not */ - private static boolean isGroupByEqualsAfterGroupByEliminate(Set queryGroupShuttledExpression, + private static boolean isGroupByEqualsAfterGroupByEliminate(Set queryGroupByShuttledExpression, Map viewShuttledExpressionQueryBasedToGroupByExpressionMap, Map groupByExpressionToViewShuttledExpressionQueryBasedMap, LogicalAggregate viewAggregate, CascadesContext cascadesContext) { - List projects = new ArrayList<>(); - // construct projects query used by view group expressions - for (Expression expression : queryGroupShuttledExpression) { + List viewProjects = new ArrayList<>(); + // construct viewProjects query used by view group expressions + for (Expression expression : queryGroupByShuttledExpression) { Expression chosenExpression = viewShuttledExpressionQueryBasedToGroupByExpressionMap.get(expression); if (chosenExpression == null) { return false; } - projects.add(chosenExpression instanceof NamedExpression + viewProjects.add(chosenExpression instanceof NamedExpression ? (NamedExpression) chosenExpression : new Alias(chosenExpression)); } - LogicalProject> project = new LogicalProject<>(projects, viewAggregate); + LogicalProject> viewProject = new LogicalProject<>(viewProjects, viewAggregate); // try to eliminate view group by expression which is not in query group by expression Plan rewrittenPlan = MaterializedViewUtils.rewriteByRules(cascadesContext, childContext -> { Rewriter.getCteChildrenRewriter(childContext, ImmutableList.of(Rewriter.topDown(new EliminateGroupByKey()))).execute(); return childContext.getRewritePlan(); - }, project, project); + }, viewProject, viewProject); - Optional> aggreagateOptional = + Optional> viewAggreagateOptional = rewrittenPlan.collectFirst(LogicalAggregate.class::isInstance); - if (!aggreagateOptional.isPresent()) { + if (!viewAggreagateOptional.isPresent()) { return false; } // check result after view group by eliminate by functional dependency - List viewEliminatedGroupByExpressions = aggreagateOptional.get().getGroupByExpressions(); - if (viewEliminatedGroupByExpressions.size() != queryGroupShuttledExpression.size()) { + List viewEliminatedGroupByExpressions = viewAggreagateOptional.get().getGroupByExpressions(); + if (viewEliminatedGroupByExpressions.size() != queryGroupByShuttledExpression.size()) { return false; } Set viewGroupShuttledExpressionQueryBased = new HashSet<>(); - for (Expression viewExpression : aggreagateOptional.get().getGroupByExpressions()) { + for (Expression viewExpression : viewAggreagateOptional.get().getGroupByExpressions()) { Expression viewExpressionQueryBased = groupByExpressionToViewShuttledExpressionQueryBasedMap.get(viewExpression); if (viewExpressionQueryBased == null) { @@ -584,7 +579,7 @@ private static boolean isGroupByEqualsAfterGroupByEliminate(Set quer } viewGroupShuttledExpressionQueryBased.add(viewExpressionQueryBased); } - return queryGroupShuttledExpression.equals(viewGroupShuttledExpressionQueryBased); + return queryGroupByShuttledExpression.equals(viewGroupShuttledExpressionQueryBased); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateOnNoneAggregateRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateOnNoneAggregateRule.java index 7107238a3092195..45514f6fb15e6ef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateOnNoneAggregateRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateOnNoneAggregateRule.java @@ -124,7 +124,7 @@ protected Plan rewriteQueryByView(MatchMode matchMode, StructInfo queryStructInf () -> String.format("query aggregate = %s", queryAggregate.treeString())); return null; } - return doRewriteQueryByView(queryStructInfo, + return aggregateRewriteByView(queryStructInfo, viewToQuerySlotMapping, queryTopPlanAndAggPair, tempRewritedPlan, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/rollup/AggFunctionRollUpHandler.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/rollup/AggFunctionRollUpHandler.java index a96c272521aef71..190080424511c74 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/rollup/AggFunctionRollUpHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/rollup/AggFunctionRollUpHandler.java @@ -62,12 +62,12 @@ public abstract Function doRollup( /** * Extract the function arguments by functionWithAny pattern * Such as functionWithAny def is bitmap_union(to_bitmap(Any.INSTANCE)), - * actualFunction is bitmap_union(to_bitmap(case when a = 5 then 1 else 2 end)) + * actualExpression is bitmap_union(to_bitmap(case when a = 5 then 1 else 2 end)) * after extracting, the return argument is: case when a = 5 then 1 else 2 end */ - protected static List extractArguments(Expression functionWithAny, Function actualFunction) { + protected static List extractArguments(Expression functionWithAny, Expression actualExpression) { Set exprSetToRemove = functionWithAny.collectToSet(expr -> !(expr instanceof Any)); - return actualFunction.collectFirst(expr -> + return actualExpression.collectFirst(expr -> exprSetToRemove.stream().noneMatch(exprToRemove -> exprToRemove.equals(expr))) .map(expr -> ImmutableList.of((Expression) expr)).orElse(ImmutableList.of()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/rollup/ContainDistinctFunctionRollupHandler.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/rollup/ContainDistinctFunctionRollupHandler.java index 4d9e6810ce4521b..63b148dfcd7698c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/rollup/ContainDistinctFunctionRollupHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/rollup/ContainDistinctFunctionRollupHandler.java @@ -76,6 +76,7 @@ public boolean canRollup(AggregateFunction queryAggregateFunction, Set mvExpressionsQueryBased = mvExprToMvScanExprQueryBased.keySet(); Set aggregateFunctionParamSlots = queryAggregateFunctionShuttled.collectToSet(Slot.class::isInstance); if (aggregateFunctionParamSlots.stream().anyMatch(slot -> !mvExpressionsQueryBased.contains(slot))) { + // If query use any slot not in view, can not roll up return false; } return true; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/rollup/DirectRollupHandler.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/rollup/DirectRollupHandler.java index fe443afbc995364..ae94eda115c681c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/rollup/DirectRollupHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/rollup/DirectRollupHandler.java @@ -52,9 +52,7 @@ public boolean canRollup( && viewExpression instanceof NullableAggregateFunction ? ((NullableAggregateFunction) queryAggregateFunctionShuttled).equalsIgnoreNullable(viewExpression) : queryAggregateFunctionShuttled.equals(viewExpression); - return isEquals && MappingRollupHandler.AGGREGATE_ROLL_UP_EQUIVALENT_FUNCTION_MAP.keySet().stream() - .noneMatch(aggFunction -> aggFunction.equals(queryAggregateFunction)) - && !(queryAggregateFunction instanceof Combinator); + return isEquals && !(queryAggregateFunction instanceof Combinator); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/rollup/MappingRollupHandler.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/rollup/MappingRollupHandler.java index f3f81235f3cfcb0..5ed661dd51a7e50 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/rollup/MappingRollupHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/rollup/MappingRollupHandler.java @@ -23,6 +23,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.AnyValue; import org.apache.doris.nereids.trees.expressions.functions.agg.BitmapUnion; import org.apache.doris.nereids.trees.expressions.functions.agg.BitmapUnionCount; import org.apache.doris.nereids.trees.expressions.functions.agg.Count; @@ -132,6 +133,9 @@ public class MappingRollupHandler extends AggFunctionRollUpHandler { AGGREGATE_ROLL_UP_EQUIVALENT_FUNCTION_MAP.put(new HllUnion(Any.INSTANCE), new HllUnion(Any.INSTANCE)); + // support roll up when any_value + AGGREGATE_ROLL_UP_EQUIVALENT_FUNCTION_MAP.put(new AnyValue(Any.INSTANCE), + Any.INSTANCE); } @Override @@ -141,24 +145,20 @@ public boolean canRollup(AggregateFunction queryAggregateFunction, Map mvExprToMvScanExprQueryBasedMap) { // handle complex functions roll up by mapping and combinator expression // eg: query is count(distinct param), mv sql is bitmap_union(to_bitmap(param)) - Expression viewExpression = mvExprToMvScanExprQueryBasedPair.key(); - if (!super.canRollup(queryAggregateFunction, queryAggregateFunctionShuttled, - mvExprToMvScanExprQueryBasedPair, mvExprToMvScanExprQueryBasedMap)) { - return false; - } - Function viewFunction = (Function) viewExpression; - for (Map.Entry> equivalentFunctionEntry : + Expression actualViewExpression = mvExprToMvScanExprQueryBasedPair.key(); + for (Map.Entry> queryToViewEquivalentMapEntry : AGGREGATE_ROLL_UP_EQUIVALENT_FUNCTION_MAP.asMap().entrySet()) { - if (equivalentFunctionEntry.getKey().equals(queryAggregateFunction)) { - // check is have equivalent function or not - for (Expression equivalentFunction : equivalentFunctionEntry.getValue()) { - if (!Any.equals(equivalentFunction, viewFunction)) { + if (queryToViewEquivalentMapEntry.getKey().equals(queryAggregateFunction)) { + for (Expression mappedViewExpression : queryToViewEquivalentMapEntry.getValue()) { + if (!Any.equals(mappedViewExpression, actualViewExpression)) { + // check the mapping view expression is equivalent with actual view expression continue; } // check param in query function is same as the view function - List viewFunctionArguments = extractArguments(equivalentFunction, viewFunction); - List queryFunctionArguments = - extractArguments(equivalentFunctionEntry.getKey(), queryAggregateFunction); + List viewFunctionArguments = extractArguments(mappedViewExpression, + actualViewExpression); + List queryFunctionArguments = extractArguments(queryToViewEquivalentMapEntry.getKey(), + queryAggregateFunction); // check argument size,we only support roll up function which has only one argument currently if (queryFunctionArguments.size() != 1 || viewFunctionArguments.size() != 1) { continue; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AnyValue.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AnyValue.java index 50cdacc132b3d40..e85c2ce551c230d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AnyValue.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AnyValue.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.coercion.AnyDataType; @@ -33,7 +34,7 @@ * AggregateFunction 'any_value'. This class is generated by GenerateFunction. */ public class AnyValue extends NullableAggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature { + implements UnaryExpression, ExplicitlyCastableSignature, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.retArgType(0).args(AnyDataType.INSTANCE_WITHOUT_INDEX) @@ -80,4 +81,14 @@ public R accept(ExpressionVisitor visitor, C context) { public List getSignatures() { return SIGNATURES; } + + @Override + public Function constructRollUp(Expression param, Expression... varParams) { + return new AnyValue(this.distinct, this.alwaysNullable, param); + } + + @Override + public boolean canRollUp() { + return true; + } } diff --git a/regression-test/data/nereids_rules_p0/mv/agg_with_roll_up/any_value_roll_up.out b/regression-test/data/nereids_rules_p0/mv/agg_with_roll_up/any_value_roll_up.out new file mode 100644 index 000000000000000..9c69bcde1ab8468 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/mv/agg_with_roll_up/any_value_roll_up.out @@ -0,0 +1,73 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !query1_0_before -- +3 o +3 o +3 o +3 o +4 o + +-- !query1_0_after -- +3 o +3 o +3 o +3 o +4 o + +-- !query2_0_before -- +3 o +3 o +3 o +3 o +4 o + +-- !query2_0_after -- +3 o +3 o +3 o +3 o +4 o + +-- !query3_0_before -- +3 3 o + +-- !query3_0_after -- +3 3 o + +-- !query4_0_before -- +3 o + +-- !query4_0_after -- +3 o + +-- !query5_0_before -- +3 3 o + +-- !query5_0_after -- +3 3 o + +-- !query5_1_before -- +3 3 o + +-- !query5_1_after -- +3 3 o + +-- !query6_0_before -- +3 o + +-- !query6_0_after -- +3 o + +-- !query6_1_before -- +3 o +3 o +3 o +3 o +4 o + +-- !query6_1_after -- +3 o +3 o +3 o +3 o +4 o + diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/any_value_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/any_value_roll_up.groovy new file mode 100644 index 000000000000000..44815c8a1427f07 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/any_value_roll_up.groovy @@ -0,0 +1,417 @@ +package mv.agg_with_roll_up +// 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("any_value_roll_up") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF"; + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + + sql """ + drop table if exists orders_2 + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders_2 ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + O_COMMENT VARCHAR(79) NOT NULL + ) + DUPLICATE KEY(o_orderkey, o_custkey) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + drop table if exists lineitem_2 + """ + + sql""" + CREATE TABLE IF NOT EXISTS lineitem_2 ( + l_orderkey INTEGER NOT NULL, + l_partkey INTEGER NOT NULL, + l_suppkey INTEGER NOT NULL, + l_linenumber INTEGER NOT NULL, + l_quantity DECIMALV3(15,2) NOT NULL, + l_extendedprice DECIMALV3(15,2) NOT NULL, + l_discount DECIMALV3(15,2) NOT NULL, + l_tax DECIMALV3(15,2) NOT NULL, + l_returnflag CHAR(1) NOT NULL, + l_linestatus CHAR(1) NOT NULL, + l_shipdate DATE NOT NULL, + l_commitdate DATE NOT NULL, + l_receiptdate DATE NOT NULL, + l_shipinstruct CHAR(25) NOT NULL, + l_shipmode CHAR(10) NOT NULL, + l_comment VARCHAR(44) NOT NULL + ) + DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """ + drop table if exists partsupp_2 + """ + + sql """ + CREATE TABLE IF NOT EXISTS partsupp_2 ( + ps_partkey INTEGER NOT NULL, + ps_suppkey INTEGER NOT NULL, + ps_availqty INTEGER NOT NULL, + ps_supplycost DECIMALV3(15,2) NOT NULL, + ps_comment VARCHAR(199) NOT NULL + ) + DUPLICATE KEY(ps_partkey, ps_suppkey) + DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """ insert into lineitem_2 values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); + """ + + sql """ + insert into orders_2 values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + sql """ + insert into partsupp_2 values + (2, 3, 9, 10.01, 'supply1'), + (2, 3, 10, 11.01, 'supply2'); + """ + + sql """analyze table partsupp_2 with sync""" + sql """analyze table lineitem_2 with sync""" + sql """analyze table orders_2 with sync""" + + + // mv has any value, query also has any value + def mv1_0 = """ + select + o_orderdate, + l_partkey, + l_suppkey, + any_value(o_orderstatus) + from lineitem_2 + left join orders_2 + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by + o_orderdate, + l_partkey, + l_suppkey; + """ + def query1_0 = """ + select + l_suppkey, + any_value(o_orderstatus) + from lineitem_2 + left join orders_2 + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by + o_orderdate, + l_suppkey; + """ + order_qt_query1_0_before "${query1_0}" + async_mv_rewrite_success(db, mv1_0, query1_0, "any_mv1_0") + order_qt_query1_0_after "${query1_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS any_mv1_0""" + + + // query has any value, mv doesn't have + def mv2_0 = """ + select + o_orderdate, + l_partkey, + l_suppkey, + o_orderstatus + from lineitem_2 + left join orders_2 + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by + o_orderstatus, + o_orderdate, + l_partkey, + l_suppkey; + """ + def query2_0 = """ + select + l_suppkey, + any_value(o_orderstatus) + from lineitem_2 + left join orders_2 + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by + o_orderdate, + l_suppkey; + """ + order_qt_query2_0_before "${query2_0}" + async_mv_rewrite_success(db, mv2_0, query2_0, "any_mv2_0") + order_qt_query2_0_after "${query2_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS any_mv2_0""" + + + // mv has any value, query also has any value, with same filter + def mv3_0 = """ + select + l_suppkey, + o_orderdate, + l_partkey, + o_orderstatus, + any_value(l_suppkey), + any_value(o_orderstatus) + from lineitem_2 + left join orders_2 + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where l_partkey = 3 and o_orderstatus = 'o' + group by + o_orderstatus, + o_orderdate, + l_partkey, + l_suppkey; + """ + def query3_0 = """ + select + l_suppkey, + any_value(l_suppkey), + any_value(o_orderstatus) + from lineitem_2 + left join orders_2 + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where l_partkey = 3 and o_orderstatus = 'o' + group by + o_orderdate, + l_suppkey; + """ + order_qt_query3_0_before "${query3_0}" + async_mv_rewrite_success(db, mv3_0, query3_0, "any_mv3_0") + order_qt_query3_0_after "${query3_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS any_mv3_0""" + + + // query has any value, mv doesn't have, with same filter + def mv4_0 = """ + select + o_orderdate, + l_partkey, + l_suppkey, + o_orderstatus + from lineitem_2 + left join orders_2 + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where l_partkey = 3 and o_orderstatus = 'o' + group by + o_orderstatus, + o_orderdate, + l_partkey, + l_suppkey; + """ + def query4_0 = """ + select + any_value(l_suppkey), + any_value(o_orderstatus) + from lineitem_2 + left join orders_2 + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where l_partkey = 3 and o_orderstatus = 'o' + group by + o_orderdate, + l_suppkey; + """ + order_qt_query4_0_before "${query4_0}" + async_mv_rewrite_success(db, mv4_0, query4_0, "any_mv4_0") + order_qt_query4_0_after "${query4_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS any_mv4_0""" + + + // mv has any value, query also has any value, with different filter + def mv5_0 = """ + select + o_orderdate, + any_value(l_partkey), + o_orderstatus, + l_suppkey, + any_value(l_suppkey), + any_value(o_orderstatus) + from lineitem_2 + left join orders_2 + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where l_partkey = 3 + group by + o_orderstatus, + o_orderdate, + l_partkey, + l_suppkey; + """ + def query5_0 = """ + select + any_value(l_partkey), + any_value(l_suppkey), + any_value(o_orderstatus) + from lineitem_2 + left join orders_2 + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where l_partkey = 3 and o_orderstatus = 'o' + group by + o_orderdate, + l_suppkey; + """ + order_qt_query5_0_before "${query5_0}" + async_mv_rewrite_success(db, mv5_0, query5_0, "any_mv5_0") + order_qt_query5_0_after "${query5_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS any_mv5_0""" + + + def mv5_1 = """ + select + o_orderdate, + any_value(l_partkey), + any_value(l_suppkey), + any_value(o_orderstatus) + from lineitem_2 + left join orders_2 + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where l_partkey = 3 and o_orderstatus = 'o' + group by + o_orderstatus, + o_orderdate, + l_partkey, + l_suppkey; + """ + def query5_1 = """ + select + l_suppkey, + any_value(l_suppkey), + any_value(o_orderstatus) + from lineitem_2 + left join orders_2 + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where l_partkey = 3 + group by + o_orderdate, + l_suppkey; + """ + order_qt_query5_1_before "${query5_1}" + async_mv_rewrite_fail(db, mv5_1, query5_1, "any_mv5_1") + order_qt_query5_1_after "${query5_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS any_mv5_1""" + + + // query has any value, mv doesn't have, with different filter + def mv6_0 = """ + select + o_orderdate, + l_partkey, + l_suppkey, + o_orderstatus + from lineitem_2 + left join orders_2 + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderstatus = 'o' + group by + o_orderstatus, + o_orderdate, + l_partkey, + l_suppkey; + """ + def query6_0 = """ + select + any_value(l_suppkey), + any_value(o_orderstatus) + from lineitem_2 + left join orders_2 + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where l_partkey = 3 and o_orderstatus = 'o' + group by + o_orderdate, + l_suppkey; + """ + order_qt_query6_0_before "${query6_0}" + async_mv_rewrite_success(db, mv6_0, query6_0, "any_mv6_0") + order_qt_query6_0_after "${query6_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS any_mv6_0""" + + + def mv6_1 = """ + select + o_orderdate, + l_partkey, + l_suppkey, + o_orderstatus + from lineitem_2 + left join orders_2 + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where l_partkey = 3 and o_orderstatus = 'o' + group by + o_orderstatus, + o_orderdate, + l_partkey, + l_suppkey; + """ + def query6_1 = """ + select + l_suppkey, + any_value(o_orderstatus) + from lineitem_2 + left join orders_2 + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderstatus = 'o' + group by + o_orderdate, + l_suppkey; + """ + order_qt_query6_1_before "${query6_1}" + async_mv_rewrite_fail(db, mv6_1, query6_1, "any_mv6_1") + order_qt_query6_1_after "${query6_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS any_mv6_1""" + +} From aa92038a55779497b0e0f5c3f4eb1805030e559a Mon Sep 17 00:00:00 2001 From: Gabriel Date: Mon, 25 Nov 2024 20:53:36 +0800 Subject: [PATCH 079/110] [fix](runtime filter) Fix unreasonable wrong status (#44545) Introduced by #43627 Query could be finished when runtime filter was sent to a global runtime filter merger. Here we should return `EOF` status instead of `InvalidArgument`. --- be/src/runtime/fragment_mgr.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index 16b75794511c06d..47b007edf5dc9bc 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -1292,8 +1292,8 @@ Status FragmentMgr::sync_filter_size(const PSyncFilterSizeRequest* request) { if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { query_ctx = q_ctx; } else { - return Status::InvalidArgument( - "Sync filter size failed: Query context (query-id: {}) not found", + return Status::EndOfFile( + "Sync filter size failed: Query context (query-id: {}) already finished", queryid.to_string()); } } @@ -1313,8 +1313,8 @@ Status FragmentMgr::merge_filter(const PMergeFilterRequest* request, if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { query_ctx = q_ctx; } else { - return Status::InvalidArgument( - "Merge filter size failed: Query context (query-id: {}) not found", + return Status::EndOfFile( + "Merge filter size failed: Query context (query-id: {}) already finished", queryid.to_string()); } } From a89926a41f05e86a2460529a74f7878c89c54fac Mon Sep 17 00:00:00 2001 From: Gabriel Date: Mon, 25 Nov 2024 20:54:12 +0800 Subject: [PATCH 080/110] =?UTF-8?q?[fix](coordinator)=20Fix=20wrong=20buck?= =?UTF-8?q?et=20assginment=20in=20old-version=20coordin=E2=80=A6=20(#44539?= =?UTF-8?q?)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …ator Follow-up for : #44459 --- .../java/org/apache/doris/qe/Coordinator.java | 95 +++++++++---------- 1 file changed, 47 insertions(+), 48 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index e508efde42dbf66..61ecaf7fc8b359a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -1433,31 +1433,8 @@ protected void computeFragmentExecParams() throws Exception { destParams.instanceExecParams.get(0).bucketSeqSet.add(0); } // process bucket shuffle join on fragment without scan node - TNetworkAddress dummyServer = new TNetworkAddress("0.0.0.0", 0); while (bucketSeq < bucketNum) { - TPlanFragmentDestination dest = new TPlanFragmentDestination(); - - dest.fragment_instance_id = new TUniqueId(-1, -1); - dest.server = dummyServer; - dest.setBrpcServer(dummyServer); - - Set hostSet = new HashSet<>(); - for (int insIdx = 0; insIdx < destParams.instanceExecParams.size(); insIdx++) { - FInstanceExecParam instanceExecParams = destParams.instanceExecParams.get(insIdx); - if (destParams.ignoreDataDistribution - && hostSet.contains(instanceExecParams.host)) { - continue; - } - hostSet.add(instanceExecParams.host); - if (instanceExecParams.bucketSeqSet.contains(bucketSeq)) { - dest.fragment_instance_id = instanceExecParams.instanceId; - dest.server = toRpcHost(instanceExecParams.host); - dest.setBrpcServer(toBrpcHost(instanceExecParams.host)); - instanceExecParams.recvrId = params.destinations.size(); - break; - } - } - + TPlanFragmentDestination dest = setDestination(destParams, params.destinations.size(), bucketSeq); bucketSeq++; params.destinations.add(dest); } @@ -1504,6 +1481,50 @@ protected void computeFragmentExecParams() throws Exception { } } + private TPlanFragmentDestination setDestination(FragmentExecParams destParams, int recvrId, int bucketSeq) + throws Exception { + TPlanFragmentDestination dest = new TPlanFragmentDestination(); + TNetworkAddress dummyServer = new TNetworkAddress("0.0.0.0", 0); + dest.fragment_instance_id = new TUniqueId(-1, -1); + dest.server = dummyServer; + dest.setBrpcServer(dummyServer); + + if (destParams.ignoreDataDistribution) { + Map>> hostToInstanceIdAndBucketSeq + = new HashMap<>(); + for (int insIdx = 0; insIdx < destParams.instanceExecParams.size(); insIdx++) { + FInstanceExecParam instanceExecParams = destParams.instanceExecParams.get(insIdx); + hostToInstanceIdAndBucketSeq.putIfAbsent(instanceExecParams.host, + Pair.of(instanceExecParams.instanceId, new HashSet<>())); + hostToInstanceIdAndBucketSeq.get(instanceExecParams.host).second.addAll( + instanceExecParams.bucketSeqSet); + } + for (int insIdx = 0; insIdx < destParams.instanceExecParams.size(); insIdx++) { + FInstanceExecParam instanceExecParams = destParams.instanceExecParams.get(insIdx); + if (hostToInstanceIdAndBucketSeq.get(instanceExecParams.host).second.contains(bucketSeq)) { + dest.fragment_instance_id = hostToInstanceIdAndBucketSeq.get(instanceExecParams.host) + .first; + dest.server = toRpcHost(instanceExecParams.host); + dest.setBrpcServer(toBrpcHost(instanceExecParams.host)); + instanceExecParams.recvrId = recvrId; + break; + } + } + } else { + for (int insIdx = 0; insIdx < destParams.instanceExecParams.size(); insIdx++) { + FInstanceExecParam instanceExecParams = destParams.instanceExecParams.get(insIdx); + if (instanceExecParams.bucketSeqSet.contains(bucketSeq)) { + dest.fragment_instance_id = instanceExecParams.instanceId; + dest.server = toRpcHost(instanceExecParams.host); + dest.setBrpcServer(toBrpcHost(instanceExecParams.host)); + instanceExecParams.recvrId = recvrId; + break; + } + } + } + return dest; + } + private void computeMultiCastFragmentParams() throws Exception { for (FragmentExecParams params : fragmentExecParamsMap.values()) { if (!(params.fragment instanceof MultiCastPlanFragment)) { @@ -1556,31 +1577,9 @@ private void computeMultiCastFragmentParams() throws Exception { destParams.instanceExecParams.get(0).bucketSeqSet.add(0); } // process bucket shuffle join on fragment without scan node - TNetworkAddress dummyServer = new TNetworkAddress("0.0.0.0", 0); while (bucketSeq < bucketNum) { - TPlanFragmentDestination dest = new TPlanFragmentDestination(); - - dest.fragment_instance_id = new TUniqueId(-1, -1); - dest.server = dummyServer; - dest.setBrpcServer(dummyServer); - - Set hostSet = new HashSet<>(); - for (int insIdx = 0; insIdx < destParams.instanceExecParams.size(); insIdx++) { - FInstanceExecParam instanceExecParams = destParams.instanceExecParams.get(insIdx); - if (destParams.ignoreDataDistribution - && hostSet.contains(instanceExecParams.host)) { - continue; - } - hostSet.add(instanceExecParams.host); - if (instanceExecParams.bucketSeqSet.contains(bucketSeq)) { - dest.fragment_instance_id = instanceExecParams.instanceId; - dest.server = toRpcHost(instanceExecParams.host); - dest.setBrpcServer(toBrpcHost(instanceExecParams.host)); - instanceExecParams.recvrId = params.destinations.size(); - break; - } - } - + TPlanFragmentDestination dest = setDestination(destParams, params.destinations.size(), + bucketSeq); bucketSeq++; destinations.add(dest); } From 9f5b49f7347341b986bcb89949ecf410d129e408 Mon Sep 17 00:00:00 2001 From: LiBinfeng Date: Mon, 25 Nov 2024 22:10:42 +0800 Subject: [PATCH 081/110] [fix](regression) add exception when can not find profile with specific tag (#44530) Related PR: #43103 Problem Summary: When getting profile of old planner, we would find that it can not get profile for some statements, which is not expected to check results of nereids planner also revert show trash command, cause of implementation problem --- .../nereids/parser/LogicalPlanBuilder.java | 7 -- .../doris/nereids/trees/plans/PlanType.java | 1 - .../plans/commands/ShowTrashCommand.java | 78 ------------------- .../trees/plans/visitor/CommandVisitor.java | 5 -- .../regression/action/ProfileAction.groovy | 5 ++ .../ddl/show_trash/test_nereids_trash.groovy | 22 ------ 6 files changed, 5 insertions(+), 113 deletions(-) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTrashCommand.java delete mode 100644 regression-test/suites/nereids_p0/ddl/show_trash/test_nereids_trash.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 5513b4f5ecbebda..5d31b284e15562e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -227,7 +227,6 @@ import org.apache.doris.nereids.DorisParser.ShowStorageEnginesContext; import org.apache.doris.nereids.DorisParser.ShowTableIdContext; import org.apache.doris.nereids.DorisParser.ShowTabletsBelongContext; -import org.apache.doris.nereids.DorisParser.ShowTrashContext; import org.apache.doris.nereids.DorisParser.ShowTriggersContext; import org.apache.doris.nereids.DorisParser.ShowVariablesContext; import org.apache.doris.nereids.DorisParser.ShowViewContext; @@ -495,7 +494,6 @@ import org.apache.doris.nereids.trees.plans.commands.ShowStorageEnginesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTabletsBelongCommand; -import org.apache.doris.nereids.trees.plans.commands.ShowTrashCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTriggersCommand; import org.apache.doris.nereids.trees.plans.commands.ShowVariablesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowViewCommand; @@ -4127,11 +4125,6 @@ public SetDefaultStorageVaultCommand visitSetDefaultStorageVault(SetDefaultStora return new SetDefaultStorageVaultCommand(stripQuotes(ctx.identifier().getText())); } - @Override - public LogicalPlan visitShowTrash(ShowTrashContext ctx) { - return new ShowTrashCommand(); - } - @Override public Object visitRefreshCatalog(RefreshCatalogContext ctx) { if (ctx.name != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 4e62a7e7269be97..f36af26e89273e3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -204,7 +204,6 @@ public enum PlanType { SHOW_ROLE_COMMAND, SHOW_STORAGE_ENGINES_COMMAND, SHOW_TABLE_ID_COMMAND, - SHOW_TRASH_COMMAND, SHOW_TRIGGERS_COMMAND, SHOW_VARIABLES_COMMAND, SHOW_AUTHORS_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTrashCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTrashCommand.java deleted file mode 100644 index 58146cb78575123..000000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTrashCommand.java +++ /dev/null @@ -1,78 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.nereids.trees.plans.commands; - -import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.Env; -import org.apache.doris.catalog.ScalarType; -import org.apache.doris.common.proc.TrashProcDir; -import org.apache.doris.nereids.trees.plans.PlanType; -import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; -import org.apache.doris.qe.ConnectContext; -import org.apache.doris.qe.ShowResultSet; -import org.apache.doris.qe.ShowResultSetMetaData; -import org.apache.doris.qe.StmtExecutor; -import org.apache.doris.system.Backend; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; - -import java.util.List; - -/** - * show trash command - */ -public class ShowTrashCommand extends ShowCommand { - private List backends = Lists.newArrayList(); - - public ShowTrashCommand() { - super(PlanType.SHOW_TRASH_COMMAND); - } - - public List getBackends() { - return backends; - } - - public ShowResultSetMetaData getMetaData() { - ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); - for (String title : TrashProcDir.TITLE_NAMES) { - builder.addColumn(new Column(title, ScalarType.createVarchar(30))); - } - return builder.build(); - } - - private ShowResultSet handleShowTrash() throws Exception { - ImmutableMap backendsInfo = Env.getCurrentSystemInfo().getAllBackendsByAllCluster(); - for (Backend backend : backendsInfo.values()) { - this.backends.add(backend); - } - List> infos = Lists.newArrayList(); - TrashProcDir.getTrashInfo(backends, infos); - return new ShowResultSet(getMetaData(), infos); - } - - @Override - public R accept(PlanVisitor visitor, C context) { - return visitor.visitShowTrashCommand(this, context); - } - - @Override - public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { - return handleShowTrash(); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 815c5c67030c343..4383ebc01a0b9e8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -86,7 +86,6 @@ import org.apache.doris.nereids.trees.plans.commands.ShowStorageEnginesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTabletsBelongCommand; -import org.apache.doris.nereids.trees.plans.commands.ShowTrashCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTriggersCommand; import org.apache.doris.nereids.trees.plans.commands.ShowVariablesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowViewCommand; @@ -419,10 +418,6 @@ default R visitShowTableIdCommand(ShowTableIdCommand showTableIdCommand, C conte return visitCommand(showTableIdCommand, context); } - default R visitShowTrashCommand(ShowTrashCommand showTrashCommand, C context) { - return visitCommand(showTrashCommand, context); - } - default R visitSyncCommand(SyncCommand syncCommand, C context) { return visitCommand(syncCommand, context); } diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/ProfileAction.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/ProfileAction.groovy index 5f6c00be9439dce..2e24ae1f7bc1678 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/ProfileAction.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/ProfileAction.groovy @@ -82,8 +82,10 @@ class ProfileAction implements SuiteAction { def jsonSlurper = new JsonSlurper() List profileData = jsonSlurper.parseText(body).data.rows + def canFindProfile = false; for (final def profileItem in profileData) { if (profileItem["Sql Statement"].toString().contains(tag)) { + canFindProfile = true def profileId = profileItem["Profile ID"].toString() def profileCli = new HttpCliAction(context) @@ -113,6 +115,9 @@ class ProfileAction implements SuiteAction { break } } + if (!canFindProfile) { + throw new IllegalStateException("Missing profile with tag: " + tag) + } } httpCli.run() } finally { diff --git a/regression-test/suites/nereids_p0/ddl/show_trash/test_nereids_trash.groovy b/regression-test/suites/nereids_p0/ddl/show_trash/test_nereids_trash.groovy deleted file mode 100644 index 5a78b27e41c72e1..000000000000000 --- a/regression-test/suites/nereids_p0/ddl/show_trash/test_nereids_trash.groovy +++ /dev/null @@ -1,22 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -suite("show_trash_nereids") { - // can not use qt command since the output change based on cluster and backend ip - checkNereidsExecute("""show trash;""") - checkNereidsExecute("""show trash on "127.0.0.1:9050";""") -} \ No newline at end of file From d1bf328c91ae91324241166a7059cbe420630612 Mon Sep 17 00:00:00 2001 From: Luwei Date: Mon, 25 Nov 2024 23:33:19 +0800 Subject: [PATCH 082/110] [improve](cloud) adjust timeout settings for BE brpc connection pool to meta-service (#44374) --- be/src/cloud/cloud_meta_mgr.cpp | 8 ++++---- be/src/cloud/config.cpp | 2 +- be/src/cloud/config.h | 4 ++-- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/be/src/cloud/cloud_meta_mgr.cpp b/be/src/cloud/cloud_meta_mgr.cpp index 5142d897475e03b..8e21498b0d873d7 100644 --- a/be/src/cloud/cloud_meta_mgr.cpp +++ b/be/src/cloud/cloud_meta_mgr.cpp @@ -243,12 +243,12 @@ class MetaServiceProxy { long deadline = now; // connection age only works without list endpoint. if (!is_meta_service_endpoint_list && - config::meta_service_connection_age_base_minutes > 0) { + config::meta_service_connection_age_base_seconds > 0) { std::default_random_engine rng(static_cast(now)); std::uniform_int_distribution<> uni( - config::meta_service_connection_age_base_minutes, - config::meta_service_connection_age_base_minutes * 2); - deadline = now + duration_cast(minutes(uni(rng))).count(); + config::meta_service_connection_age_base_seconds, + config::meta_service_connection_age_base_seconds * 2); + deadline = now + duration_cast(seconds(uni(rng))).count(); } else { deadline = LONG_MAX; } diff --git a/be/src/cloud/config.cpp b/be/src/cloud/config.cpp index 32e3250f87c2586..e95c295ae1daa56 100644 --- a/be/src/cloud/config.cpp +++ b/be/src/cloud/config.cpp @@ -28,7 +28,7 @@ DEFINE_Bool(meta_service_use_load_balancer, "false"); DEFINE_mInt32(meta_service_rpc_timeout_ms, "10000"); DEFINE_Bool(meta_service_connection_pooled, "true"); DEFINE_mInt64(meta_service_connection_pool_size, "20"); -DEFINE_mInt32(meta_service_connection_age_base_minutes, "5"); +DEFINE_mInt32(meta_service_connection_age_base_seconds, "30"); DEFINE_mInt32(meta_service_idle_connection_timeout_ms, "0"); DEFINE_mInt32(meta_service_rpc_retry_times, "200"); DEFINE_mInt32(meta_service_brpc_timeout_ms, "10000"); diff --git a/be/src/cloud/config.h b/be/src/cloud/config.h index 8af967afb8c67b0..b345e6355921a49 100644 --- a/be/src/cloud/config.h +++ b/be/src/cloud/config.h @@ -34,7 +34,7 @@ static inline bool is_cloud_mode() { // If meta services are deployed behind a load balancer, set this config to "host:port" of the load balancer. // Here is a set of configs to configure the connection behaviors: // - meta_service_connection_pooled: distribute the long connections to different RS of the VIP. -// - meta_service_connection_age_base_minutes: expire the connection after a random time during [base, 2*base], +// - meta_service_connection_age_base_seconds: expire the connection after a random time during [base, 2*base], // so that the BE has a chance to connect to a new RS. (When you add a new RS, the BE will connect to it) // - meta_service_idle_connection_timeout_ms: rebuild the idle connections after the timeout exceeds. Some LB // vendors will reset the connection if it is idle for a long time. @@ -50,7 +50,7 @@ DECLARE_mInt64(meta_service_connection_pool_size); // has a chance to connect to a new RS. Set zero to disable it. // // Only works when meta_service_endpoint is set to a single host. -DECLARE_mInt32(meta_service_connection_age_base_minutes); +DECLARE_mInt32(meta_service_connection_age_base_seconds); // Rebuild the idle connections after the timeout exceeds. Set zero to disable it. // // Only works when meta_service_endpoint is set to a single host. From 0fc0cc7383688e36d97b83ac70b59aea8130fb81 Mon Sep 17 00:00:00 2001 From: walter Date: Mon, 25 Nov 2024 23:40:44 +0800 Subject: [PATCH 083/110] [feat](binlog) Support rename rollup & partition binlogs (#44526) --- .../apache/doris/binlog/BinlogManager.java | 20 +++++++++++++++++++ .../org/apache/doris/persist/EditLog.java | 8 ++++---- gensrc/thrift/FrontendService.thrift | 6 +++--- 3 files changed, 27 insertions(+), 7 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java index 0fadfc2b542c85f..3a033c981038c29 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java @@ -337,6 +337,26 @@ public void addTableRename(TableInfo info, long commitSeq) { addBinlog(dbId, tableIds, commitSeq, timestamp, type, data, false, info); } + public void addRollupRename(TableInfo info, long commitSeq) { + long dbId = info.getDbId(); + List tableIds = Lists.newArrayList(); + tableIds.add(info.getTableId()); + long timestamp = -1; + TBinlogType type = TBinlogType.RENAME_ROLLUP; + String data = info.toJson(); + addBinlog(dbId, tableIds, commitSeq, timestamp, type, data, false, info); + } + + public void addPartitionRename(TableInfo info, long commitSeq) { + long dbId = info.getDbId(); + List tableIds = Lists.newArrayList(); + tableIds.add(info.getTableId()); + long timestamp = -1; + TBinlogType type = TBinlogType.RENAME_PARTITION; + String data = info.toJson(); + addBinlog(dbId, tableIds, commitSeq, timestamp, type, data, false, info); + } + public void addModifyComment(ModifyCommentOperationLog info, long commitSeq) { long dbId = info.getDbId(); List tableIds = Lists.newArrayList(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index 96f5d43aed2cbd8..1e70eb634b2073c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -318,7 +318,7 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { case OperationType.OP_RENAME_PARTITION: { TableInfo info = (TableInfo) journal.getData(); env.replayRenamePartition(info); - env.getBinlogManager().addTableRename(info, logId); + env.getBinlogManager().addPartitionRename(info, logId); break; } case OperationType.OP_RENAME_COLUMN: { @@ -366,7 +366,7 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { case OperationType.OP_RENAME_ROLLUP: { TableInfo info = (TableInfo) journal.getData(); env.replayRenameRollup(info); - env.getCurrentEnv().getBinlogManager().addTableRename(info, logId); + env.getBinlogManager().addRollupRename(info, logId); break; } case OperationType.OP_LOAD_START: @@ -1591,13 +1591,13 @@ public void logModifyViewDef(AlterViewInfo alterViewInfo) { public void logRollupRename(TableInfo tableInfo) { long logId = logEdit(OperationType.OP_RENAME_ROLLUP, tableInfo); LOG.info("log rollup rename, logId : {}, infos: {}", logId, tableInfo); - Env.getCurrentEnv().getBinlogManager().addTableRename(tableInfo, logId); + Env.getCurrentEnv().getBinlogManager().addRollupRename(tableInfo, logId); } public void logPartitionRename(TableInfo tableInfo) { long logId = logEdit(OperationType.OP_RENAME_PARTITION, tableInfo); LOG.info("log partition rename, logId : {}, infos: {}", logId, tableInfo); - Env.getCurrentEnv().getBinlogManager().addTableRename(tableInfo, logId); + Env.getCurrentEnv().getBinlogManager().addPartitionRename(tableInfo, logId); } public void logColumnRename(TableRenameColumnInfo info) { diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index c79931fe8855aab..fc7f98e16e3f746 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -1194,6 +1194,8 @@ enum TBinlogType { REPLACE_TABLE = 18, MODIFY_TABLE_ADD_OR_DROP_INVERTED_INDICES = 19, INDEX_CHANGE_JOB = 20, + RENAME_ROLLUP = 21, + RENAME_PARTITION = 22, // Keep some IDs for allocation so that when new binlog types are added in the // future, the changes can be picked back to the old versions without breaking @@ -1210,9 +1212,7 @@ enum TBinlogType { // MODIFY_XXX = 17, // MIN_UNKNOWN = 18, // UNKNOWN_3 = 19, - MIN_UNKNOWN = 21, - UNKNOWN_6 = 22, - UNKNOWN_7 = 23, + MIN_UNKNOWN = 23, UNKNOWN_8 = 24, UNKNOWN_9 = 25, UNKNOWN_10 = 26, From 9867ba3f3f96620f28a39123382432f8654483c9 Mon Sep 17 00:00:00 2001 From: Gavin Chou Date: Mon, 25 Nov 2024 23:41:07 +0800 Subject: [PATCH 084/110] [opt](file_cache) Add config to enable base compaction output to file cache (#44497) Previous implementation does not allow the output of base compaction write into file cache, which may have some performance penalty. This commit add a config to make that policy configurable. be.conf `enable_file_cache_keep_base_compaction_output` it is false by default. If your file cache is ample enough to accommodate all the data in your database, enable this option; otherwise, it is recommended to leave it disabled. --- be/src/common/config.cpp | 5 +++-- be/src/common/config.h | 11 +++++++++-- be/src/olap/compaction.cpp | 8 ++++++-- 3 files changed, 18 insertions(+), 6 deletions(-) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 2938e81a25eb01f..0cffa30cdca4434 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -959,8 +959,6 @@ DEFINE_Int32(doris_remote_scanner_thread_pool_thread_num, "48"); // number of s3 scanner thread pool queue size DEFINE_Int32(doris_remote_scanner_thread_pool_queue_size, "102400"); DEFINE_mInt64(block_cache_wait_timeout_ms, "1000"); -DEFINE_mInt64(cache_lock_long_tail_threshold, "1000"); -DEFINE_Int64(file_cache_recycle_keys_size, "1000000"); // limit the queue of pending batches which will be sent by a single nodechannel DEFINE_mInt64(nodechannel_pending_queue_max_bytes, "67108864"); @@ -1054,6 +1052,9 @@ DEFINE_Bool(enable_ttl_cache_evict_using_lru, "true"); DEFINE_mBool(enbale_dump_error_file, "true"); // limit the max size of error log on disk DEFINE_mInt64(file_cache_error_log_limit_bytes, "209715200"); // 200MB +DEFINE_mInt64(cache_lock_long_tail_threshold, "1000"); +DEFINE_Int64(file_cache_recycle_keys_size, "1000000"); +DEFINE_mBool(enable_file_cache_keep_base_compaction_output, "false"); DEFINE_mInt32(index_cache_entry_stay_time_after_lookup_s, "1800"); DEFINE_mInt32(inverted_index_cache_stale_sweep_time_sec, "600"); diff --git a/be/src/common/config.h b/be/src/common/config.h index e6247f596a177cb..caee1f320c103e5 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1011,8 +1011,6 @@ DECLARE_mInt64(nodechannel_pending_queue_max_bytes); // The batch size for sending data by brpc streaming client DECLARE_mInt64(brpc_streaming_client_batch_bytes); DECLARE_mInt64(block_cache_wait_timeout_ms); -DECLARE_mInt64(cache_lock_long_tail_threshold); -DECLARE_Int64(file_cache_recycle_keys_size); DECLARE_Bool(enable_brpc_builtin_services); @@ -1095,6 +1093,15 @@ DECLARE_Bool(enable_ttl_cache_evict_using_lru); DECLARE_mBool(enbale_dump_error_file); // limit the max size of error log on disk DECLARE_mInt64(file_cache_error_log_limit_bytes); +DECLARE_mInt64(cache_lock_long_tail_threshold); +DECLARE_Int64(file_cache_recycle_keys_size); +// Base compaction may retrieve and produce some less frequently accessed data, +// potentially affecting the file cache hit rate. +// This configuration determines whether to retain the output within the file cache. +// Make your choice based on the following considerations: +// If your file cache is ample enough to accommodate all the data in your database, +// enable this option; otherwise, it is recommended to leave it disabled. +DECLARE_mBool(enable_file_cache_keep_base_compaction_output); // inverted index searcher cache // cache entry stay time after lookup diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index d707349132036c1..68ed0322a9ec926 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -1190,8 +1190,12 @@ Status CloudCompactionMixin::construct_output_rowset_writer(RowsetWriterContext& ctx.compaction_level = _engine.cumu_compaction_policy(compaction_policy) ->new_compaction_level(_input_rowsets); } - - ctx.write_file_cache = compaction_type() == ReaderType::READER_CUMULATIVE_COMPACTION; + // We presume that the data involved in cumulative compaction is sufficiently 'hot' + // and should always be retained in the cache. + // TODO(gavin): Ensure that the retention of hot data is implemented with precision. + ctx.write_file_cache = (compaction_type() == ReaderType::READER_CUMULATIVE_COMPACTION) || + (config::enable_file_cache_keep_base_compaction_output && + compaction_type() == ReaderType::READER_BASE_COMPACTION); ctx.file_cache_ttl_sec = _tablet->ttl_seconds(); _output_rs_writer = DORIS_TRY(_tablet->create_rowset_writer(ctx, _is_vertical)); RETURN_IF_ERROR(_engine.meta_mgr().prepare_rowset(*_output_rs_writer->rowset_meta().get())); From 322edf6d0962f14a3c882e799701a072ecbd5938 Mon Sep 17 00:00:00 2001 From: starocean999 Date: Tue, 26 Nov 2024 10:02:25 +0800 Subject: [PATCH 085/110] [fix](test)set replication_allocation = 1 (#44199) --- .../tablet_prune/test_tablet_prune.groovy | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/regression-test/suites/nereids_rules_p0/tablet_prune/test_tablet_prune.groovy b/regression-test/suites/nereids_rules_p0/tablet_prune/test_tablet_prune.groovy index a87571a73405136..b1dd19e92d07a27 100644 --- a/regression-test/suites/nereids_rules_p0/tablet_prune/test_tablet_prune.groovy +++ b/regression-test/suites/nereids_rules_p0/tablet_prune/test_tablet_prune.groovy @@ -21,7 +21,13 @@ suite("test_tablet_prune") { sql "drop table if exists t_customers_wide_index" sql """ - CREATE TABLE `t_customers_wide_index` ( `CUSTOMER_ID` int NULL, `ADDRESS` varchar(1500) NULL) ENGINE=OLAP UNIQUE KEY(`CUSTOMER_ID`) DISTRIBUTED BY HASH(`CUSTOMER_ID`) BUCKETS 32 PROPERTIES ( "file_cache_ttl_seconds" = "0", "is_being_synced" = "false", "storage_medium" = "hdd", "storage_format" = "V2", "inverted_index_storage_format" = "V2", "enable_unique_key_merge_on_write" = "true", "light_schema_change" = "true", "store_row_column" = "true", "row_store_page_size" = "16384", "disable_auto_compaction" = "false", "enable_single_replica_compaction" = "false", "group_commit_interval_ms" = "10000", "group_commit_data_bytes" = "134217728", "enable_mow_light_delete" = "false" ); """ + CREATE TABLE `t_customers_wide_index` + (`CUSTOMER_ID` int NULL,`ADDRESS` varchar(1500) NULL) + ENGINE=OLAP + UNIQUE KEY(`CUSTOMER_ID`) + DISTRIBUTED BY HASH(`CUSTOMER_ID`) + BUCKETS 32 + PROPERTIES ( "replication_allocation" = "tag.location.default: 1");""" sql """ insert into t_customers_wide_index values (1, "111"); """ From 14d928b7206b1f4cbbe2580cdb7e38a206a99894 Mon Sep 17 00:00:00 2001 From: starocean999 Date: Tue, 26 Nov 2024 10:04:36 +0800 Subject: [PATCH 086/110] [fix](test)fix unstable case (#44401) --- .../nereids_p0/ddl/alter/test_nereids_role.groovy | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/regression-test/suites/nereids_p0/ddl/alter/test_nereids_role.groovy b/regression-test/suites/nereids_p0/ddl/alter/test_nereids_role.groovy index e7ffedd3b4f033b..f23982188269b7e 100644 --- a/regression-test/suites/nereids_p0/ddl/alter/test_nereids_role.groovy +++ b/regression-test/suites/nereids_p0/ddl/alter/test_nereids_role.groovy @@ -17,10 +17,10 @@ import org.junit.Assert; -suite("test_nereids_role", "account") { - def role= 'nereids_account_role_test' - def user = 'nereids_acount_role_user_test' - def dbName = 'nereids_account_role_test_db' +suite("test_nereids_role_x", "account") { + def role= 'nereids_account_role_test_x' + def user = 'nereids_acount_role_user_test_x' + def dbName = 'nereids_account_role_test_db_x' def pwd = 'C123_567p' try_sql("DROP ROLE ${role}") @@ -28,8 +28,8 @@ suite("test_nereids_role", "account") { sql """DROP DATABASE IF EXISTS ${dbName}""" sql """CREATE DATABASE IF NOT EXISTS ${dbName}""" sql """CREATE ROLE ${role}""" - sql """GRANT SELECT_PRIV ON ${context.config.defaultDb} TO ROLE '${role}'""" - sql """GRANT SELECT_PRIV ON ${dbName} TO ROLE '${role}'""" + sql """GRANT SELECT_PRIV ON internal.${context.config.defaultDb}.* TO ROLE '${role}'""" + sql """GRANT SELECT_PRIV ON internal.${dbName}.* TO ROLE '${role}'""" sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}' DEFAULT ROLE '${role}'""" def result1 = connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { sql "show databases like '${dbName}'" From dba889493171dfdd686b44f420022a74dc52183b Mon Sep 17 00:00:00 2001 From: Mingxi <71588583+joker-star-l@users.noreply.github.com> Date: Tue, 26 Nov 2024 10:43:35 +0800 Subject: [PATCH 087/110] [fix](logstash plugin) fix group_commit and sprintf with missing value (#44329) --- .../logstash/lib/logstash/outputs/doris.rb | 44 ++++++++----- .../logstash/lib/logstash/util/formater.rb | 65 +++++++++++++++++++ .../logstash/logstash-output-doris.gemspec | 6 +- 3 files changed, 94 insertions(+), 21 deletions(-) create mode 100644 extension/logstash/lib/logstash/util/formater.rb diff --git a/extension/logstash/lib/logstash/outputs/doris.rb b/extension/logstash/lib/logstash/outputs/doris.rb index c57b7cb47b2b298..02e7591b0a354bc 100644 --- a/extension/logstash/lib/logstash/outputs/doris.rb +++ b/extension/logstash/lib/logstash/outputs/doris.rb @@ -22,8 +22,8 @@ require "logstash/namespace" require "logstash/json" require "logstash/util/shortname_resolver" +require 'logstash/util/formater' require "uri" -require "logstash/plugin_mixins/http_client" require "securerandom" require "json" require "base64" @@ -77,9 +77,9 @@ class LogStash::Outputs::Doris < LogStash::Outputs::Base def print_plugin_info() - @@plugins = Gem::Specification.find_all{|spec| spec.name =~ /logstash-output-doris/ } - @plugin_name = @@plugins[0].name - @plugin_version = @@plugins[0].version + @plugins = Gem::Specification.find_all{|spec| spec.name =~ /logstash-output-doris/ } + @plugin_name = @plugins[0].name + @plugin_version = @plugins[0].version @logger.debug("Running #{@plugin_name} version #{@plugin_version}") @logger.info("Initialized doris output with settings", @@ -90,17 +90,17 @@ def print_plugin_info() end def register - @http_query = "/api/#{db}/#{table}/_stream_load" + @http_query = "/api/#{@db}/#{@table}/_stream_load" @hostnames_pool = - parse_http_hosts(http_hosts, + parse_http_hosts(@http_hosts, ShortNameResolver.new(ttl: @host_resolve_ttl_sec, logger: @logger)) @request_headers = make_request_headers @logger.info("request headers: ", @request_headers) @group_commit = false - if http_headers.has_key?("group_commit") && http_headers["group_commit"] != "off_mode" + if @request_headers.has_key?("group_commit") && @request_headers["group_commit"] != "off_mode" @group_commit = true end @logger.info("group_commit: ", @group_commit) @@ -196,7 +196,7 @@ def send_events(events) http_headers = @request_headers.dup if !@group_commit # only set label if group_commit is off_mode or not set, since lable can not be used with group_commit - http_headers["label"] = label_prefix + "_" + @db + "_" + @table + "_" + Time.now.strftime('%Y%m%d_%H%M%S_%L_' + SecureRandom.uuid) + http_headers["label"] = @label_prefix + "_" + @db + "_" + @table + "_" + Time.now.strftime('%Y%m%d_%H%M%S_%L_' + SecureRandom.uuid) end req_count = 0 @@ -211,7 +211,15 @@ def send_events(events) rescue => e @logger.warn("doris stream load response: #{response} is not a valid JSON") end - if response_json["Status"] == "Success" + + status = response_json["Status"] + + if status == 'Label Already Exists' + @logger.warn("Label already exists: #{response_json['Label']}, skip #{event_num} records.") + break + end + + if status == "Success" || status == "Publish Timeout" @total_bytes.addAndGet(documents.size) @total_rows.addAndGet(event_num) break @@ -221,7 +229,7 @@ def send_events(events) if @max_retries >= 0 && req_count > @max_retries @logger.warn("DROP this batch after failed #{req_count} times.") if @save_on_failure - @logger.warn("Try save to disk.Disk file path : #{save_dir}/#{table}_#{save_file}") + @logger.warn("Try save to disk.Disk file path : #{@save_dir}/#{@table}_#{@save_file}") save_to_disk(documents) end break @@ -252,13 +260,13 @@ def make_request(documents, http_headers, hosts, query, host = "") response = "" begin - response = RestClient.put(url, documents, http_headers) { |response, request, result| - case response.code + response = RestClient.put(url, documents, http_headers) { |res, request, result| + case res.code when 301, 302, 307 - @logger.debug("redirect to: #{response.headers[:location]}") - response.follow_redirection + @logger.debug("redirect to: #{res.headers[:location]}") + res.follow_redirection else - response.return! + res.return! end } rescue => e @@ -304,14 +312,14 @@ def convert_mapping(mapping, event) elsif mapping.is_a?(Array) mapping.map { |elem| convert_mapping(elem, event) } else - event.sprintf(mapping) + Formater.sprintf(event, mapping) end end private def save_to_disk(documents) begin - file = File.open("#{save_dir}/#{db}_#{table}_#{save_file}", "a") + file = File.open("#{@save_dir}/#{@db}_#{@table}_#{@save_file}", "a") file.write(documents) rescue IOError => e log_failure("An error occurred while saving file to disk: #{e}", @@ -330,7 +338,7 @@ def make_request_headers() headers = @headers || {} headers["Expect"] ||= "100-continue" headers["Content-Type"] ||= "text/plain;charset=utf-8" - headers["Authorization"] = "Basic " + Base64.strict_encode64("#{user}:#{password.value}") + headers["Authorization"] = "Basic " + Base64.strict_encode64("#{@user}:#{@password.value}") headers end diff --git a/extension/logstash/lib/logstash/util/formater.rb b/extension/logstash/lib/logstash/util/formater.rb new file mode 100644 index 000000000000000..8c3540d73473f99 --- /dev/null +++ b/extension/logstash/lib/logstash/util/formater.rb @@ -0,0 +1,65 @@ +# 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. + +require 'java' + +class Formater + + # copied form https://www.rubydoc.info/gems/logstash-event/LogStash/Event#sprintf-instance_method + # modified by doris: return empty str if template field does not exist in the event rather than the template. + def self.sprintf(event, format) + format = format.to_s + if format.index("%").nil? + return format + end + + format.gsub(/%\{[^}]+}/) do |tok| + # Take the inside of the %{ ... } + key = tok[2...-1] + + if key == "+%s" + # Got %{+%s}, support for unix epoch time + next event.timestamp.to_i + elsif key[0, 1] == "+" + t = event.timestamp + formatter = org.joda.time.format.DateTimeFormat.forPattern(key[1..-1]) \ + .withZone(org.joda.time.DateTimeZone::UTC) + # next org.joda.time.Instant.new(t.tv_sec * 1000 + t.tv_usec / 1000).toDateTime.toString(formatter) + # Invoke a specific Instant constructor to avoid this warning in JRuby + # > ambiguous Java methods found, using org.joda.time.Instant(long) + # org.joda.time.Instant.java_class.constructor(Java::long).new_instance( + # t.tv_sec * 1000 + t.tv_usec / 1000 + # ).to_java.toDateTime.toString(formatter) + mill = java.lang.Long.valueOf(t.to_i * 1000 + t.tv_usec / 1000) + org.joda.time.Instant.new(mill).toDateTime.toString(formatter) + else + value = event.get(key) + case value + when nil + '' # empty str if this field does not exist in this event + when Array + value.join(",") # join by ',' if value is an array + when Hash + value.to_json # convert hashes to json + else + value # otherwise return the value + end # case value + end # 'key' checking + end # format.gsub... + end + +end diff --git a/extension/logstash/logstash-output-doris.gemspec b/extension/logstash/logstash-output-doris.gemspec index 6536d89ccc3f841..689b93503f6a8b2 100644 --- a/extension/logstash/logstash-output-doris.gemspec +++ b/extension/logstash/logstash-output-doris.gemspec @@ -41,7 +41,7 @@ Gem::Specification.new do |s| s.add_runtime_dependency 'mini_cache', ">= 1.0.0", "< 2.0.0" s.add_runtime_dependency "rest-client", '~> 2.1' - s.add_development_dependency 'logstash-devutils', '~> 2.0', '>= 2.0.3' - s.add_development_dependency 'sinatra', '~> 2.0', '>= 2.0.8.1' - s.add_development_dependency 'webrick', '~> 1.6' + s.add_development_dependency 'logstash-devutils', '~> 1.3' + s.add_development_dependency 'sinatra', '~> 1.4' + s.add_development_dependency 'webrick', '~> 1.9' end From 530bee05c215ec2f013760f8a520513dd7486591 Mon Sep 17 00:00:00 2001 From: LiBinfeng Date: Tue, 26 Nov 2024 10:57:57 +0800 Subject: [PATCH 088/110] [fix](cases) fix regression test set and unset command failed by accident (#44569) Related PR: #43103 When set and unset commands set the same variable, it would failed by accident duplicate case added by related pr --- .../variable_p0/set_and_unset_variable.out | 193 ------------------ .../variable_p0/set_and_unset_variable.groovy | 112 ---------- 2 files changed, 305 deletions(-) delete mode 100644 regression-test/data/variable_p0/set_and_unset_variable.out delete mode 100644 regression-test/suites/variable_p0/set_and_unset_variable.groovy diff --git a/regression-test/data/variable_p0/set_and_unset_variable.out b/regression-test/data/variable_p0/set_and_unset_variable.out deleted file mode 100644 index 506542ea754557d..000000000000000 --- a/regression-test/data/variable_p0/set_and_unset_variable.out +++ /dev/null @@ -1,193 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !cmd -- -0 - --- !cmd -- -0 - --- !cmd -- -0 - --- !cmd -- -wait_timeout 1000 28800 1 - --- !cmd -- -0 - --- !cmd -- -wait_timeout 28800 28800 0 - --- !cmd -- -0 - --- !cmd -- -runtime_filter_type BLOOM_FILTER IN_OR_BLOOM_FILTER,MIN_MAX 1 - --- !cmd -- -runtime_filter_type IN_OR_BLOOM_FILTER,MIN_MAX IN_OR_BLOOM_FILTER,MIN_MAX 0 - --- !cmd -- -0 - --- !cmd -- -runtime_filter_type IN_OR_BLOOM_FILTER,MIN_MAX IN_OR_BLOOM_FILTER,MIN_MAX 0 - --- !cmd -- -runtime_filter_type IN_OR_BLOOM_FILTER,MIN_MAX IN_OR_BLOOM_FILTER,MIN_MAX 0 - --- !cmd -- -0 - --- !cmd -- -runtime_filter_type BLOOM_FILTER IN_OR_BLOOM_FILTER,MIN_MAX 1 - --- !cmd -- -runtime_filter_type BLOOM_FILTER IN_OR_BLOOM_FILTER,MIN_MAX 1 - --- !cmd -- -0 - --- !cmd -- -runtime_filter_type IN_OR_BLOOM_FILTER,MIN_MAX IN_OR_BLOOM_FILTER,MIN_MAX 0 - --- !cmd -- -runtime_filter_type IN_OR_BLOOM_FILTER,MIN_MAX IN_OR_BLOOM_FILTER,MIN_MAX 0 - --- !cmd -- -0 - --- !cmd -- -experimental_enable_agg_state true false 1 - --- !cmd -- -experimental_enable_agg_state false false 0 - --- !cmd -- -0 - --- !cmd -- -experimental_enable_agg_state false false 0 - --- !cmd -- -experimental_enable_agg_state false false 0 - --- !cmd -- -0 - --- !cmd -- -experimental_enable_agg_state true false 1 - --- !cmd -- -experimental_enable_agg_state true false 1 - --- !cmd -- -0 - --- !cmd -- -experimental_enable_agg_state false false 0 - --- !cmd -- -experimental_enable_agg_state false false 0 - --- !cmd -- -0 - --- !cmd -- -deprecated_enable_local_exchange false true 1 - --- !cmd -- -deprecated_enable_local_exchange true true 0 - --- !cmd -- -0 - --- !cmd -- -deprecated_enable_local_exchange true true 0 - --- !cmd -- -deprecated_enable_local_exchange true true 0 - --- !cmd -- -0 - --- !cmd -- -0 - --- !cmd -- -0 - --- !cmd -- -0 - --- !cmd -- -0 - --- !cmd -- -runtime_filter_type IN_OR_BLOOM_FILTER,MIN_MAX IN_OR_BLOOM_FILTER,MIN_MAX 0 - --- !cmd -- -experimental_enable_agg_state false false 0 - --- !cmd -- -deprecated_enable_local_exchange true true 0 - --- !cmd -- -show_hidden_columns false false 0 - --- !cmd -- -show_hidden_columns false false 0 - --- !cmd -- -0 - --- !cmd -- -0 - --- !cmd -- -0 - --- !cmd -- -0 - --- !cmd -- -0 - --- !cmd -- -runtime_filter_type IN_OR_BLOOM_FILTER,MIN_MAX IN_OR_BLOOM_FILTER,MIN_MAX 0 - --- !cmd -- -experimental_enable_agg_state false false 0 - --- !cmd -- -deprecated_enable_local_exchange true true 0 - --- !cmd -- -show_hidden_columns false false 0 - --- !cmd -- -show_hidden_columns false false 0 - --- !cmd -- -read_only true true 0 - --- !cmd -- -0 - --- !cmd -- -read_only true true 0 - --- !cmd -- -read_only true true 0 - --- !cmd -- -super_read_only true true 0 - --- !cmd -- -0 - --- !cmd -- -super_read_only true true 0 - --- !cmd -- -super_read_only true true 0 - diff --git a/regression-test/suites/variable_p0/set_and_unset_variable.groovy b/regression-test/suites/variable_p0/set_and_unset_variable.groovy deleted file mode 100644 index f29a56bd6d01321..000000000000000 --- a/regression-test/suites/variable_p0/set_and_unset_variable.groovy +++ /dev/null @@ -1,112 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -suite("set_and_unset_variable", "nonConcurrent") { - qt_cmd """UNSET VARIABLE ALL""" - qt_cmd """UNSET global VARIABLE ALL""" - - qt_cmd """set wait_timeout = 1000""" - qt_cmd """show variables like 'wait_timeout'""" - qt_cmd """UNSET VARIABLE wait_timeout""" - qt_cmd """show variables like 'wait_timeout'""" - - qt_cmd """set runtime_filter_type='BLOOM_FILTER'""" - qt_cmd """show session variables like 'runtime_filter_type'""" - qt_cmd """show global variables like 'runtime_filter_type'""" - qt_cmd """UNSET VARIABLE runtime_filter_type""" - qt_cmd """show session variables like 'runtime_filter_type'""" - qt_cmd """show global variables like 'runtime_filter_type'""" - - qt_cmd """set global runtime_filter_type='BLOOM_FILTER'""" - qt_cmd """show session variables like 'runtime_filter_type'""" - qt_cmd """show global variables like 'runtime_filter_type'""" - qt_cmd """UNSET global VARIABLE runtime_filter_type""" - qt_cmd """show session variables like 'runtime_filter_type'""" - qt_cmd """show global variables like 'runtime_filter_type'""" - - // test variables with experimental_ prefix in session scope - qt_cmd """set experimental_enable_agg_state='true'""" - qt_cmd """show session variables like 'experimental_enable_agg_state'""" - qt_cmd """show global variables like 'experimental_enable_agg_state'""" - qt_cmd """UNSET VARIABLE experimental_enable_agg_state""" - qt_cmd """show session variables like 'experimental_enable_agg_state'""" - qt_cmd """show global variables like 'experimental_enable_agg_state'""" - - // test variables with experimental_ prefix in global scope - qt_cmd """set global experimental_enable_agg_state='true'""" - qt_cmd """show session variables like 'experimental_enable_agg_state'""" - qt_cmd """show global variables like 'experimental_enable_agg_state'""" - qt_cmd """UNSET global VARIABLE experimental_enable_agg_state""" - qt_cmd """show session variables like 'experimental_enable_agg_state'""" - qt_cmd """show global variables like 'experimental_enable_agg_state'""" - - // test variables with deprecated_ prefix - qt_cmd """set deprecated_enable_local_exchange = false""" - qt_cmd """show session variables like 'deprecated_enable_local_exchange'""" - qt_cmd """show global variables like 'deprecated_enable_local_exchange'""" - qt_cmd """UNSET global VARIABLE deprecated_enable_local_exchange""" - qt_cmd """show session variables like 'deprecated_enable_local_exchange'""" - qt_cmd """show global variables like 'deprecated_enable_local_exchange'""" - - // test UNSET VARIABLE ALL - qt_cmd """set runtime_filter_type='BLOOM_FILTER'""" - qt_cmd """set experimental_enable_agg_state='true'""" - qt_cmd """set deprecated_enable_local_exchange = false""" - qt_cmd """set show_hidden_columns=true""" - qt_cmd """UNSET VARIABLE ALL""" - qt_cmd """show session variables like 'runtime_filter_type'""" - qt_cmd """show session variables like 'experimental_enable_agg_state'""" - qt_cmd """show session variables like 'deprecated_enable_local_exchange'""" - qt_cmd """show session variables like 'show_hidden_columns'""" - - qt_cmd """select * from information_schema.session_variables where variable_name = 'show_hidden_columns'""" - - // test UNSET GLOBAL VARIABLE ALL - qt_cmd """set global runtime_filter_type='BLOOM_FILTER'""" - qt_cmd """set global experimental_enable_agg_state='true'""" - qt_cmd """set global deprecated_enable_local_exchange = false""" - qt_cmd """set show_hidden_columns=true""" - qt_cmd """UNSET global VARIABLE ALL""" - qt_cmd """show global variables like 'runtime_filter_type'""" - qt_cmd """show global variables like 'experimental_enable_agg_state'""" - qt_cmd """show global variables like 'deprecated_enable_local_exchange'""" - qt_cmd """show global variables like 'show_hidden_columns'""" - - qt_cmd """select * from information_schema.global_variables where variable_name = 'show_hidden_columns'""" - - // test read_only - qt_cmd """show variables like 'read_only'""" - test { - sql "set read_only=true" - exception "should be set with SET GLOBAL" - } - qt_cmd "set global read_only=true" - qt_cmd """show global variables like 'read_only'""" - qt_cmd """show variables like 'read_only'""" - sql "set global read_only=false" - - // test super_read_only - qt_cmd """show variables like 'super_read_only'""" - test { - sql "set super_read_only=true" - exception "should be set with SET GLOBAL" - } - qt_cmd "set global super_read_only=true" - qt_cmd """show global variables like 'super_read_only'""" - qt_cmd """show variables like 'super_read_only'""" - sql "set global super_read_only=false" -} From ea6cd589696f548db16d6ee4a18375cbc46e7252 Mon Sep 17 00:00:00 2001 From: HappenLee Date: Tue, 26 Nov 2024 11:16:30 +0800 Subject: [PATCH 089/110] [Refactor](vec) Remove the unless api in datatype (#44513) Remove the unless api in datatype --- be/src/vec/common/columns_hashing.h | 3 +-- be/src/vec/common/hash_table/hash_key_type.h | 4 ++-- be/src/vec/core/block.cpp | 2 +- be/src/vec/data_types/data_type.cpp | 16 -------------- be/src/vec/data_types/data_type.h | 21 ------------------- be/src/vec/data_types/data_type_array.h | 1 - be/src/vec/data_types/data_type_bitmap.h | 1 - be/src/vec/data_types/data_type_decimal.h | 1 - .../data_type_fixed_length_object.h | 1 - be/src/vec/data_types/data_type_hll.h | 1 - be/src/vec/data_types/data_type_jsonb.h | 1 - be/src/vec/data_types/data_type_map.h | 1 - be/src/vec/data_types/data_type_nothing.h | 1 - be/src/vec/data_types/data_type_nullable.h | 5 ++--- be/src/vec/data_types/data_type_number_base.h | 1 - be/src/vec/data_types/data_type_object.h | 2 -- .../vec/data_types/data_type_quantilestate.h | 1 - be/src/vec/data_types/data_type_string.h | 1 - be/src/vec/data_types/data_type_struct.cpp | 8 ------- be/src/vec/data_types/data_type_struct.h | 2 -- 20 files changed, 6 insertions(+), 68 deletions(-) diff --git a/be/src/vec/common/columns_hashing.h b/be/src/vec/common/columns_hashing.h index 00c86686d2b03ae..f0a365cfc09f373 100644 --- a/be/src/vec/common/columns_hashing.h +++ b/be/src/vec/common/columns_hashing.h @@ -40,8 +40,7 @@ using Sizes = std::vector; inline Sizes get_key_sizes(const std::vector& data_types) { Sizes key_sizes; for (const auto& data_type : data_types) { - key_sizes.emplace_back(data_type->get_maximum_size_of_value_in_memory() - - data_type->is_nullable()); + key_sizes.emplace_back(data_type->get_size_of_value_in_memory() - data_type->is_nullable()); } return key_sizes; } diff --git a/be/src/vec/common/hash_table/hash_key_type.h b/be/src/vec/common/hash_table/hash_key_type.h index 2c14e4ab687f87b..c6f4c8addf30c39 100644 --- a/be/src/vec/common/hash_table/hash_key_type.h +++ b/be/src/vec/common/hash_table/hash_key_type.h @@ -78,7 +78,7 @@ inline HashKeyType get_hash_key_type_fixed(const std::vectorhave_maximum_size_of_value()) { return HashKeyType::serialized; } - key_byte_size += data_type->get_maximum_size_of_value_in_memory(); + key_byte_size += data_type->get_size_of_value_in_memory(); if (data_type->is_nullable()) { has_null = true; key_byte_size--; @@ -106,7 +106,7 @@ inline HashKeyType get_hash_key_type(const std::vector& throw Exception(ErrorCode::INTERNAL_ERROR, "meet invalid type, type={}", t->get_name()); } - size_t size = t->get_maximum_size_of_value_in_memory(); + size_t size = t->get_size_of_value_in_memory(); if (size == sizeof(vectorized::UInt8)) { return HashKeyType::int8_key; } else if (size == sizeof(vectorized::UInt16)) { diff --git a/be/src/vec/core/block.cpp b/be/src/vec/core/block.cpp index 6083be0287738e6..4dc553b1a5790f6 100644 --- a/be/src/vec/core/block.cpp +++ b/be/src/vec/core/block.cpp @@ -812,7 +812,7 @@ void Block::filter_block_internal(Block* block, const std::vector& col if (column->size() != count) { if (column->is_exclusive()) { const auto result_size = column->assume_mutable()->filter(filter); - if (result_size != count) { + if (result_size != count) [[unlikely]] { throw Exception(ErrorCode::INTERNAL_ERROR, "result_size not equal with filter_size, result_size={}, " "filter_size={}", diff --git a/be/src/vec/data_types/data_type.cpp b/be/src/vec/data_types/data_type.cpp index 28415076ba339a3..d950c7509cef85d 100644 --- a/be/src/vec/data_types/data_type.cpp +++ b/be/src/vec/data_types/data_type.cpp @@ -55,22 +55,6 @@ String IDataType::do_get_name() const { return get_family_name(); } -void IDataType::update_avg_value_size_hint(const IColumn& column, double& avg_value_size_hint) { - /// Update the average value size hint if amount of read rows isn't too small - size_t row_size = column.size(); - if (row_size > 10) { - double current_avg_value_size = - static_cast(column.byte_size()) / static_cast(row_size); - - /// Heuristic is chosen so that avg_value_size_hint increases rapidly but decreases slowly. - if (current_avg_value_size > avg_value_size_hint) { - avg_value_size_hint = std::min(1024., current_avg_value_size); /// avoid overestimation - } else if (current_avg_value_size * 2 < avg_value_size_hint) { - avg_value_size_hint = (current_avg_value_size + avg_value_size_hint * 3) / 4; - } - } -} - ColumnPtr IDataType::create_column_const(size_t size, const Field& field) const { auto column = create_column(); column->reserve(1); diff --git a/be/src/vec/data_types/data_type.h b/be/src/vec/data_types/data_type.h index 8796e2292a5c1be..86f7cf36fa8dc79 100644 --- a/be/src/vec/data_types/data_type.h +++ b/be/src/vec/data_types/data_type.h @@ -123,14 +123,6 @@ class IDataType : private boost::noncopyable { /// Checks that two instances belong to the same type virtual bool equals(const IDataType& rhs) const = 0; - /// Various properties on behaviour of data type. - - /** The data type is dependent on parameters and types with different parameters are different. - * Examples: FixedString(N), Tuple(T1, T2), Nullable(T). - * Otherwise all instances of the same class are the same types. - */ - virtual bool get_is_parametric() const = 0; - /** The data type is dependent on parameters and at least one of them is another type. * Examples: Tuple(T1, T2), Nullable(T). But FixedString(N) is not. */ @@ -180,22 +172,12 @@ class IDataType : private boost::noncopyable { return false; } - virtual bool is_value_unambiguously_represented_in_fixed_size_contiguous_memory_region() const { - return is_value_represented_by_number(); - } - /** Example: numbers, Date, DateTime, FixedString, Enum... Nullable and Tuple of such types. * Counterexamples: String, Array. * It's Ok to return false for AggregateFunction despite the fact that some of them have fixed size state. */ virtual bool have_maximum_size_of_value() const { return false; } - /** Size in amount of bytes in memory. Throws an exception if not have_maximum_size_of_value. - */ - virtual size_t get_maximum_size_of_value_in_memory() const { - return get_size_of_value_in_memory(); - } - /** Throws an exception if value is not of fixed size. */ virtual size_t get_size_of_value_in_memory() const; @@ -210,9 +192,6 @@ class IDataType : private boost::noncopyable { /// Strings, Numbers, Date, DateTime, Nullable virtual bool can_be_inside_low_cardinality() const { return false; } - /// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column. - static void update_avg_value_size_hint(const IColumn& column, double& avg_value_size_hint); - virtual int64_t get_uncompressed_serialized_bytes(const IColumn& column, int be_exec_version) const = 0; virtual char* serialize(const IColumn& column, char* buf, int be_exec_version) const = 0; diff --git a/be/src/vec/data_types/data_type_array.h b/be/src/vec/data_types/data_type_array.h index 8fdc714fec1ed61..7899715ed24defd 100644 --- a/be/src/vec/data_types/data_type_array.h +++ b/be/src/vec/data_types/data_type_array.h @@ -85,7 +85,6 @@ class DataTypeArray final : public IDataType { bool equals(const IDataType& rhs) const override; - bool get_is_parametric() const override { return true; } bool have_subtypes() const override { return true; } bool text_can_contain_only_valid_utf8() const override { return nested->text_can_contain_only_valid_utf8(); diff --git a/be/src/vec/data_types/data_type_bitmap.h b/be/src/vec/data_types/data_type_bitmap.h index 9a16732821a3a2f..24f1e6334431c1f 100644 --- a/be/src/vec/data_types/data_type_bitmap.h +++ b/be/src/vec/data_types/data_type_bitmap.h @@ -73,7 +73,6 @@ class DataTypeBitMap : public IDataType { int be_exec_version) const override; MutableColumnPtr create_column() const override; - bool get_is_parametric() const override { return false; } bool have_subtypes() const override { return false; } bool should_align_right_in_pretty_formats() const override { return false; } bool text_can_contain_only_valid_utf8() const override { return true; } diff --git a/be/src/vec/data_types/data_type_decimal.h b/be/src/vec/data_types/data_type_decimal.h index c34d88035aab953..a44390cd58d86a7 100644 --- a/be/src/vec/data_types/data_type_decimal.h +++ b/be/src/vec/data_types/data_type_decimal.h @@ -215,7 +215,6 @@ class DataTypeDecimal final : public IDataType { MutableColumnPtr create_column() const override; bool equals(const IDataType& rhs) const override; - bool get_is_parametric() const override { return true; } bool have_subtypes() const override { return false; } bool should_align_right_in_pretty_formats() const override { return true; } bool text_can_contain_only_valid_utf8() const override { return true; } diff --git a/be/src/vec/data_types/data_type_fixed_length_object.h b/be/src/vec/data_types/data_type_fixed_length_object.h index af923ddce18cf05..8f4d7a593739e07 100644 --- a/be/src/vec/data_types/data_type_fixed_length_object.h +++ b/be/src/vec/data_types/data_type_fixed_length_object.h @@ -78,7 +78,6 @@ class DataTypeFixedLengthObject final : public IDataType { int be_exec_version) const override; MutableColumnPtr create_column() const override; - bool get_is_parametric() const override { return false; } bool have_subtypes() const override { return false; } bool can_be_inside_low_cardinality() const override { return false; } diff --git a/be/src/vec/data_types/data_type_hll.h b/be/src/vec/data_types/data_type_hll.h index fb94b6d5ebb569d..2c4061cf7b4abd5 100644 --- a/be/src/vec/data_types/data_type_hll.h +++ b/be/src/vec/data_types/data_type_hll.h @@ -65,7 +65,6 @@ class DataTypeHLL : public IDataType { int be_exec_version) const override; MutableColumnPtr create_column() const override; - bool get_is_parametric() const override { return false; } bool have_subtypes() const override { return false; } bool should_align_right_in_pretty_formats() const override { return false; } bool text_can_contain_only_valid_utf8() const override { return true; } diff --git a/be/src/vec/data_types/data_type_jsonb.h b/be/src/vec/data_types/data_type_jsonb.h index 4aec1ccc0417a9d..2afd893c8d129df 100644 --- a/be/src/vec/data_types/data_type_jsonb.h +++ b/be/src/vec/data_types/data_type_jsonb.h @@ -86,7 +86,6 @@ class DataTypeJsonb final : public IDataType { bool equals(const IDataType& rhs) const override; - bool get_is_parametric() const override { return false; } bool have_subtypes() const override { return false; } bool is_comparable() const override { return false; } bool is_value_unambiguously_represented_in_contiguous_memory_region() const override { diff --git a/be/src/vec/data_types/data_type_map.h b/be/src/vec/data_types/data_type_map.h index 1715c75edeeff7e..8a3613dfe1b9daa 100644 --- a/be/src/vec/data_types/data_type_map.h +++ b/be/src/vec/data_types/data_type_map.h @@ -84,7 +84,6 @@ class DataTypeMap final : public IDataType { } bool equals(const IDataType& rhs) const override; - bool get_is_parametric() const override { return true; } bool have_subtypes() const override { return true; } bool is_comparable() const override { return key_type->is_comparable() && value_type->is_comparable(); diff --git a/be/src/vec/data_types/data_type_nothing.h b/be/src/vec/data_types/data_type_nothing.h index bc779bccb0d18a0..bb0e095b5a5a364 100644 --- a/be/src/vec/data_types/data_type_nothing.h +++ b/be/src/vec/data_types/data_type_nothing.h @@ -66,7 +66,6 @@ class DataTypeNothing final : public IDataType { bool equals(const IDataType& rhs) const override; - bool get_is_parametric() const override { return false; } bool text_can_contain_only_valid_utf8() const override { return true; } bool have_maximum_size_of_value() const override { return true; } size_t get_size_of_value_in_memory() const override { return 0; } diff --git a/be/src/vec/data_types/data_type_nullable.h b/be/src/vec/data_types/data_type_nullable.h index feba155a627c79c..d366fb5373ba05d 100644 --- a/be/src/vec/data_types/data_type_nullable.h +++ b/be/src/vec/data_types/data_type_nullable.h @@ -92,7 +92,6 @@ class DataTypeNullable final : public IDataType { return nested_data_type->is_value_unambiguously_represented_in_contiguous_memory_region(); } - bool get_is_parametric() const override { return true; } bool have_subtypes() const override { return true; } bool should_align_right_in_pretty_formats() const override { return nested_data_type->should_align_right_in_pretty_formats(); @@ -104,8 +103,8 @@ class DataTypeNullable final : public IDataType { bool have_maximum_size_of_value() const override { return nested_data_type->have_maximum_size_of_value(); } - size_t get_maximum_size_of_value_in_memory() const override { - return 1 + nested_data_type->get_maximum_size_of_value_in_memory(); + size_t get_size_of_value_in_memory() const override { + return 1 + nested_data_type->get_size_of_value_in_memory(); } bool is_nullable() const override { return true; } bool can_be_inside_low_cardinality() const override { diff --git a/be/src/vec/data_types/data_type_number_base.h b/be/src/vec/data_types/data_type_number_base.h index a376b458f5133bc..d5ddfe07e831cad 100644 --- a/be/src/vec/data_types/data_type_number_base.h +++ b/be/src/vec/data_types/data_type_number_base.h @@ -140,7 +140,6 @@ class DataTypeNumberBase : public IDataType { int be_exec_version) const override; MutableColumnPtr create_column() const override; - bool get_is_parametric() const override { return false; } bool have_subtypes() const override { return false; } bool should_align_right_in_pretty_formats() const override { return true; } bool text_can_contain_only_valid_utf8() const override { return true; } diff --git a/be/src/vec/data_types/data_type_object.h b/be/src/vec/data_types/data_type_object.h index ec60cde9f92fca6..7723d1c53774add 100644 --- a/be/src/vec/data_types/data_type_object.h +++ b/be/src/vec/data_types/data_type_object.h @@ -67,8 +67,6 @@ class DataTypeObject : public IDataType { MutableColumnPtr create_column() const override { return ColumnObject::create(is_nullable); } bool is_object() const override { return true; } bool equals(const IDataType& rhs) const override; - bool hasNullableSubcolumns() const { return is_nullable; } - bool get_is_parametric() const override { return true; } bool have_subtypes() const override { return true; }; int64_t get_uncompressed_serialized_bytes(const IColumn& column, int be_exec_version) const override; diff --git a/be/src/vec/data_types/data_type_quantilestate.h b/be/src/vec/data_types/data_type_quantilestate.h index 031c25d5b439962..5edfe123dca6df3 100644 --- a/be/src/vec/data_types/data_type_quantilestate.h +++ b/be/src/vec/data_types/data_type_quantilestate.h @@ -68,7 +68,6 @@ class DataTypeQuantileState : public IDataType { int be_exec_version) const override; MutableColumnPtr create_column() const override; - bool get_is_parametric() const override { return false; } bool have_subtypes() const override { return false; } bool should_align_right_in_pretty_formats() const override { return false; } bool text_can_contain_only_valid_utf8() const override { return true; } diff --git a/be/src/vec/data_types/data_type_string.h b/be/src/vec/data_types/data_type_string.h index dd9371686114718..8b5ea94151f1eb6 100644 --- a/be/src/vec/data_types/data_type_string.h +++ b/be/src/vec/data_types/data_type_string.h @@ -80,7 +80,6 @@ class DataTypeString : public IDataType { bool equals(const IDataType& rhs) const override; - bool get_is_parametric() const override { return false; } bool have_subtypes() const override { return false; } bool is_comparable() const override { return true; } bool is_value_unambiguously_represented_in_contiguous_memory_region() const override { diff --git a/be/src/vec/data_types/data_type_struct.cpp b/be/src/vec/data_types/data_type_struct.cpp index 6e2803e514ea171..780882379ca50cc 100644 --- a/be/src/vec/data_types/data_type_struct.cpp +++ b/be/src/vec/data_types/data_type_struct.cpp @@ -427,14 +427,6 @@ bool DataTypeStruct::is_comparable() const { [](auto&& elem) { return elem->is_comparable(); }); } -size_t DataTypeStruct::get_maximum_size_of_value_in_memory() const { - size_t res = 0; - for (const auto& elem : elems) { - res += elem->get_maximum_size_of_value_in_memory(); - } - return res; -} - size_t DataTypeStruct::get_size_of_value_in_memory() const { size_t res = 0; for (const auto& elem : elems) { diff --git a/be/src/vec/data_types/data_type_struct.h b/be/src/vec/data_types/data_type_struct.h index cd3f1b6df158457..9a1255c2c95fd71 100644 --- a/be/src/vec/data_types/data_type_struct.h +++ b/be/src/vec/data_types/data_type_struct.h @@ -103,12 +103,10 @@ class DataTypeStruct final : public IDataType { bool equals(const IDataType& rhs) const override; - bool get_is_parametric() const override { return true; } bool have_subtypes() const override { return !elems.empty(); } bool is_comparable() const override; bool text_can_contain_only_valid_utf8() const override; bool have_maximum_size_of_value() const override; - size_t get_maximum_size_of_value_in_memory() const override; size_t get_size_of_value_in_memory() const override; const DataTypePtr& get_element(size_t i) const { return elems[i]; } From 680c5729330d9200f1ba22c72d744993b2cafd46 Mon Sep 17 00:00:00 2001 From: zclllhhjj Date: Tue, 26 Nov 2024 11:16:53 +0800 Subject: [PATCH 090/110] [Fix](insert) Fix ENABLE_AUTO_CREATE_WHEN_OVERWRITE forward to Master (#44570) Problem Summary: When insert overwrite auto detect statement execute by follower, it's will not set ENABLE_AUTO_CREATE_WHEN_OVERWRITE correctly because the insert stmt will forward to master to execute by the variable will not forward. not fixed it. ### Release note (no release note for 2.1 needed because the base function has not been released now) for 3.0: Fix insert overwrite auto detect failed executing by follower. --- .../src/main/java/org/apache/doris/qe/SessionVariable.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index bd724af54c49e87..e15133246190f9b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -2309,7 +2309,7 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { @VariableMgr.VarAttr(name = ENABLE_COOLDOWN_REPLICA_AFFINITY, needForward = true) public boolean enableCooldownReplicaAffinity = true; - @VariableMgr.VarAttr(name = ENABLE_AUTO_CREATE_WHEN_OVERWRITE, description = { + @VariableMgr.VarAttr(name = ENABLE_AUTO_CREATE_WHEN_OVERWRITE, needForward = true, description = { "开启后对自动分区表的 insert overwrite 操作会对没有找到分区的插入数据按自动分区规则创建分区,默认关闭", "The insert overwrite operation on an auto-partitioned table will create partitions for inserted data" + " for which no partition is found according to the auto-partitioning rules, which is turned off" From 137ed95c340420813844fe78cb834fecfdad5148 Mon Sep 17 00:00:00 2001 From: Pxl Date: Tue, 26 Nov 2024 11:17:08 +0800 Subject: [PATCH 091/110] [Chore](runtime-filter) refactor of runtime filter void type data usage (#43957) refactor of runtime filter void type data usage --- be/src/exprs/hybrid_set.h | 84 ++---- be/src/exprs/minmax_predicate.h | 23 +- be/src/exprs/runtime_filter.cpp | 284 +----------------- be/src/exprs/runtime_filter_convertor.h | 76 +++++ be/src/pipeline/exec/hashjoin_build_sink.cpp | 3 +- .../org/apache/doris/qe/SessionVariable.java | 2 + .../data/datatype_p0/ip/ip_rf/test_ip_rf.out | 30 ++ .../datatype_p0/ip/ip_rf/test_ip_rf.groovy | 13 +- .../nereids_p0/join/test_join_on.groovy | 6 +- 9 files changed, 172 insertions(+), 349 deletions(-) create mode 100644 be/src/exprs/runtime_filter_convertor.h diff --git a/be/src/exprs/hybrid_set.h b/be/src/exprs/hybrid_set.h index 6536ec2430fe08f..dbf31a54772ea3b 100644 --- a/be/src/exprs/hybrid_set.h +++ b/be/src/exprs/hybrid_set.h @@ -17,21 +17,8 @@ #pragma once -#include - -#include - -#include "common/exception.h" -#include "common/object_pool.h" -#include "common/status.h" #include "exprs/runtime_filter.h" -#include "runtime/decimalv2_value.h" -#include "runtime/define_primitive_type.h" -#include "runtime/primitive_type.h" -#include "vec/columns/column_nullable.h" -#include "vec/columns/column_string.h" -#include "vec/common/hash_table/phmap_fwd_decl.h" -#include "vec/common/string_ref.h" +#include "exprs/runtime_filter_convertor.h" namespace doris { @@ -221,30 +208,19 @@ class HybridSetBase : public RuntimeFilterFuncBase { virtual bool find(const void* data, size_t) const = 0; virtual void find_batch(const doris::vectorized::IColumn& column, size_t rows, - doris::vectorized::ColumnUInt8::Container& results) { - LOG(FATAL) << "HybridSetBase not support find_batch"; - __builtin_unreachable(); - } - + doris::vectorized::ColumnUInt8::Container& results) = 0; virtual void find_batch_negative(const doris::vectorized::IColumn& column, size_t rows, - doris::vectorized::ColumnUInt8::Container& results) { - LOG(FATAL) << "HybridSetBase not support find_batch_negative"; - __builtin_unreachable(); - } - + doris::vectorized::ColumnUInt8::Container& results) = 0; virtual void find_batch_nullable(const doris::vectorized::IColumn& column, size_t rows, const doris::vectorized::NullMap& null_map, - doris::vectorized::ColumnUInt8::Container& results) { - LOG(FATAL) << "HybridSetBase not support find_batch_nullable"; - __builtin_unreachable(); - } + doris::vectorized::ColumnUInt8::Container& results) = 0; - virtual void find_batch_nullable_negative(const doris::vectorized::IColumn& column, size_t rows, - const doris::vectorized::NullMap& null_map, - doris::vectorized::ColumnUInt8::Container& results) { - LOG(FATAL) << "HybridSetBase not support find_batch_nullable_negative"; - __builtin_unreachable(); - } + virtual void find_batch_nullable_negative( + const doris::vectorized::IColumn& column, size_t rows, + const doris::vectorized::NullMap& null_map, + doris::vectorized::ColumnUInt8::Container& results) = 0; + + virtual void to_pb(PInFilter* filter) = 0; class IteratorBase { public: @@ -261,26 +237,6 @@ class HybridSetBase : public RuntimeFilterFuncBase { bool _contains_null = false; }; -template -const Type* check_and_get_hybrid_set(const HybridSetBase& column) { - return typeid_cast(&column); -} - -template -const Type* check_and_get_hybrid_set(const HybridSetBase* column) { - return typeid_cast(column); -} - -template -bool check_hybrid_set(const HybridSetBase& column) { - return check_and_get_hybrid_set(&column); -} - -template -bool check_hybrid_set(const HybridSetBase* column) { - return check_and_get_hybrid_set(column); -} - template ::CppType>, typename _ColumnType = typename PrimitiveTypeTraits::ColumnType> @@ -409,6 +365,14 @@ class HybridSet : public HybridSetBase { ContainerType* get_inner_set() { return &_set; } + void set_pb(PInFilter* filter, auto f) { + for (auto v : _set) { + f(filter->add_values(), v); + } + } + + void to_pb(PInFilter* filter) override { set_pb(filter, get_convertor()); } + private: ContainerType _set; ObjectPool _pool; @@ -569,6 +533,14 @@ class StringSet : public HybridSetBase { ContainerType* get_inner_set() { return &_set; } + void set_pb(PInFilter* filter, auto f) { + for (const auto& v : _set) { + f(filter->add_values(), v); + } + } + + void to_pb(PInFilter* filter) override { set_pb(filter, get_convertor()); } + private: ContainerType _set; ObjectPool _pool; @@ -735,6 +707,10 @@ class StringValueSet : public HybridSetBase { ContainerType* get_inner_set() { return &_set; } + void to_pb(PInFilter* filter) override { + throw Exception(ErrorCode::INTERNAL_ERROR, "StringValueSet do not support to_pb"); + } + private: ContainerType _set; ObjectPool _pool; diff --git a/be/src/exprs/minmax_predicate.h b/be/src/exprs/minmax_predicate.h index 377b33696c82b9d..6c5d8a2d3c4bd54 100644 --- a/be/src/exprs/minmax_predicate.h +++ b/be/src/exprs/minmax_predicate.h @@ -17,16 +17,8 @@ #pragma once -#include - -#include "common/object_pool.h" #include "exprs/runtime_filter.h" -#include "runtime/type_limit.h" -#include "vec/columns/column.h" -#include "vec/columns/column_nullable.h" -#include "vec/columns/column_string.h" -#include "vec/common/assert_cast.h" -#include "vec/common/string_ref.h" +#include "exprs/runtime_filter_convertor.h" namespace doris { // only used in Runtime Filter @@ -45,6 +37,8 @@ class MinMaxFuncBase : public RuntimeFilterFuncBase { void set_contain_null() { _contain_null = true; } + virtual void to_pb(PMinMaxFilter* filter) = 0; + protected: bool _contain_null = false; }; @@ -165,6 +159,17 @@ class MinMaxNumFunc : public MinMaxFuncBase { return Status::OK(); } + void set_pb(PMinMaxFilter* filter, auto f) { + if constexpr (NeedMin) { + f(filter->mutable_min_val(), _min); + } + if constexpr (NeedMax) { + f(filter->mutable_max_val(), _max); + } + } + + void to_pb(PMinMaxFilter* filter) override { set_pb(filter, get_convertor()); } + protected: T _max = type_limit::min(); T _min = type_limit::max(); diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index bac14b616b2ce65..6eed49e8567e1c9 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -903,17 +903,10 @@ class RuntimePredicateWrapper { return Status::InternalError("not support!"); } - HybridSetBase::IteratorBase* get_in_filter_iterator() { return _context->hybrid_set->begin(); } - void get_bloom_filter_desc(char** data, int* filter_length) { _context->bloom_filter_func->get_data(data, filter_length); } - void get_minmax_filter_desc(void** min_data, void** max_data) { - *min_data = _context->minmax_func->get_min(); - *max_data = _context->minmax_func->get_max(); - } - PrimitiveType column_type() { return _column_return_type; } bool is_bloomfilter() const { return get_real_type() == RuntimeFilterType::BLOOM_FILTER; } @@ -1560,17 +1553,6 @@ Status IRuntimeFilter::merge_from(const RuntimePredicateWrapper* wrapper) { return Status::OK(); } -template -void batch_copy(PInFilter* filter, HybridSetBase::IteratorBase* it, - void (*set_func)(PColumnValue*, const T*)) { - while (it->has_next()) { - const void* void_value = it->get_value(); - auto origin_value = reinterpret_cast(void_value); - set_func(filter->add_values(), origin_value); - it->next(); - } -} - template Status IRuntimeFilter::serialize_impl(T* request, void** data, int* len) { auto real_runtime_filter_type = _wrapper->get_real_type(); @@ -1598,273 +1580,13 @@ Status IRuntimeFilter::serialize_impl(T* request, void** data, int* len) { } void IRuntimeFilter::to_protobuf(PInFilter* filter) { - auto column_type = _wrapper->column_type(); - filter->set_column_type(to_proto(column_type)); - - auto* it = _wrapper->get_in_filter_iterator(); - DCHECK(it != nullptr); - - switch (column_type) { - case TYPE_BOOLEAN: { - batch_copy(filter, it, [](PColumnValue* column, const bool* value) { - column->set_boolval(*value); - }); - return; - } - case TYPE_TINYINT: { - batch_copy(filter, it, [](PColumnValue* column, const int8_t* value) { - column->set_intval(*value); - }); - return; - } - case TYPE_SMALLINT: { - batch_copy(filter, it, [](PColumnValue* column, const int16_t* value) { - column->set_intval(*value); - }); - return; - } - case TYPE_INT: { - batch_copy(filter, it, [](PColumnValue* column, const int32_t* value) { - column->set_intval(*value); - }); - return; - } - case TYPE_BIGINT: { - batch_copy(filter, it, [](PColumnValue* column, const int64_t* value) { - column->set_longval(*value); - }); - return; - } - case TYPE_LARGEINT: { - batch_copy(filter, it, [](PColumnValue* column, const int128_t* value) { - column->set_stringval(LargeIntValue::to_string(*value)); - }); - return; - } - case TYPE_FLOAT: { - batch_copy(filter, it, [](PColumnValue* column, const float* value) { - column->set_doubleval(*value); - }); - return; - } - case TYPE_DOUBLE: { - batch_copy(filter, it, [](PColumnValue* column, const double* value) { - column->set_doubleval(*value); - }); - return; - } - case TYPE_DATEV2: { - batch_copy>( - filter, it, [](PColumnValue* column, const DateV2Value* value) { - column->set_intval(*reinterpret_cast(value)); - }); - return; - } - case TYPE_DATETIMEV2: { - batch_copy>( - filter, it, - [](PColumnValue* column, const DateV2Value* value) { - column->set_longval(*reinterpret_cast(value)); - }); - return; - } - case TYPE_DATE: - case TYPE_DATETIME: { - batch_copy(filter, it, - [](PColumnValue* column, const VecDateTimeValue* value) { - char convert_buffer[30]; - value->to_string(convert_buffer); - column->set_stringval(convert_buffer); - }); - return; - } - case TYPE_DECIMALV2: { - batch_copy(filter, it, - [](PColumnValue* column, const DecimalV2Value* value) { - column->set_stringval(value->to_string()); - }); - return; - } - case TYPE_DECIMAL32: { - batch_copy(filter, it, [](PColumnValue* column, const int32_t* value) { - column->set_intval(*value); - }); - return; - } - case TYPE_DECIMAL64: { - batch_copy(filter, it, [](PColumnValue* column, const int64_t* value) { - column->set_longval(*value); - }); - return; - } - case TYPE_DECIMAL128I: { - batch_copy(filter, it, [](PColumnValue* column, const int128_t* value) { - column->set_stringval(LargeIntValue::to_string(*value)); - }); - return; - } - case TYPE_DECIMAL256: { - batch_copy(filter, it, [](PColumnValue* column, const wide::Int256* value) { - column->set_stringval(wide::to_string(*value)); - }); - return; - } - case TYPE_CHAR: - case TYPE_VARCHAR: - case TYPE_STRING: { - //const void* void_value = it->get_value(); - //Now the get_value return void* is StringRef - batch_copy(filter, it, [](PColumnValue* column, const StringRef* value) { - column->set_stringval(value->to_string()); - }); - return; - } - case TYPE_IPV4: { - batch_copy(filter, it, [](PColumnValue* column, const IPv4* value) { - column->set_intval(*reinterpret_cast(value)); - }); - return; - } - case TYPE_IPV6: { - batch_copy(filter, it, [](PColumnValue* column, const IPv6* value) { - column->set_stringval(LargeIntValue::to_string(*value)); - }); - return; - } - default: { - throw Exception(ErrorCode::INTERNAL_ERROR, - "runtime filter meet invalid PrimitiveType type {}", int(column_type)); - } - } + filter->set_column_type(to_proto(_wrapper->column_type())); + _wrapper->_context->hybrid_set->to_pb(filter); } void IRuntimeFilter::to_protobuf(PMinMaxFilter* filter) { - void* min_data = nullptr; - void* max_data = nullptr; - _wrapper->get_minmax_filter_desc(&min_data, &max_data); - DCHECK(min_data != nullptr && max_data != nullptr); filter->set_column_type(to_proto(_wrapper->column_type())); - - switch (_wrapper->column_type()) { - case TYPE_BOOLEAN: { - filter->mutable_min_val()->set_boolval(*reinterpret_cast(min_data)); - filter->mutable_max_val()->set_boolval(*reinterpret_cast(max_data)); - return; - } - case TYPE_TINYINT: { - filter->mutable_min_val()->set_intval(*reinterpret_cast(min_data)); - filter->mutable_max_val()->set_intval(*reinterpret_cast(max_data)); - return; - } - case TYPE_SMALLINT: { - filter->mutable_min_val()->set_intval(*reinterpret_cast(min_data)); - filter->mutable_max_val()->set_intval(*reinterpret_cast(max_data)); - return; - } - case TYPE_INT: { - filter->mutable_min_val()->set_intval(*reinterpret_cast(min_data)); - filter->mutable_max_val()->set_intval(*reinterpret_cast(max_data)); - return; - } - case TYPE_BIGINT: { - filter->mutable_min_val()->set_longval(*reinterpret_cast(min_data)); - filter->mutable_max_val()->set_longval(*reinterpret_cast(max_data)); - return; - } - case TYPE_LARGEINT: { - filter->mutable_min_val()->set_stringval( - LargeIntValue::to_string(*reinterpret_cast(min_data))); - filter->mutable_max_val()->set_stringval( - LargeIntValue::to_string(*reinterpret_cast(max_data))); - return; - } - case TYPE_FLOAT: { - filter->mutable_min_val()->set_doubleval(*reinterpret_cast(min_data)); - filter->mutable_max_val()->set_doubleval(*reinterpret_cast(max_data)); - return; - } - case TYPE_DOUBLE: { - filter->mutable_min_val()->set_doubleval(*reinterpret_cast(min_data)); - filter->mutable_max_val()->set_doubleval(*reinterpret_cast(max_data)); - return; - } - case TYPE_DATEV2: { - filter->mutable_min_val()->set_intval(*reinterpret_cast(min_data)); - filter->mutable_max_val()->set_intval(*reinterpret_cast(max_data)); - return; - } - case TYPE_DATETIMEV2: { - filter->mutable_min_val()->set_longval(*reinterpret_cast(min_data)); - filter->mutable_max_val()->set_longval(*reinterpret_cast(max_data)); - return; - } - case TYPE_DATE: - case TYPE_DATETIME: { - char convert_buffer[30]; - reinterpret_cast(min_data)->to_string(convert_buffer); - filter->mutable_min_val()->set_stringval(convert_buffer); - reinterpret_cast(max_data)->to_string(convert_buffer); - filter->mutable_max_val()->set_stringval(convert_buffer); - return; - } - case TYPE_DECIMALV2: { - filter->mutable_min_val()->set_stringval( - reinterpret_cast(min_data)->to_string()); - filter->mutable_max_val()->set_stringval( - reinterpret_cast(max_data)->to_string()); - return; - } - case TYPE_DECIMAL32: { - filter->mutable_min_val()->set_intval(*reinterpret_cast(min_data)); - filter->mutable_max_val()->set_intval(*reinterpret_cast(max_data)); - return; - } - case TYPE_DECIMAL64: { - filter->mutable_min_val()->set_longval(*reinterpret_cast(min_data)); - filter->mutable_max_val()->set_longval(*reinterpret_cast(max_data)); - return; - } - case TYPE_DECIMAL128I: { - filter->mutable_min_val()->set_stringval( - LargeIntValue::to_string(*reinterpret_cast(min_data))); - filter->mutable_max_val()->set_stringval( - LargeIntValue::to_string(*reinterpret_cast(max_data))); - return; - } - case TYPE_DECIMAL256: { - filter->mutable_min_val()->set_stringval( - wide::to_string(*reinterpret_cast(min_data))); - filter->mutable_max_val()->set_stringval( - wide::to_string(*reinterpret_cast(max_data))); - return; - } - case TYPE_CHAR: - case TYPE_VARCHAR: - case TYPE_STRING: { - const auto* min_string_value = reinterpret_cast(min_data); - filter->mutable_min_val()->set_stringval(*min_string_value); - const auto* max_string_value = reinterpret_cast(max_data); - filter->mutable_max_val()->set_stringval(*max_string_value); - break; - } - case TYPE_IPV4: { - filter->mutable_min_val()->set_intval(*reinterpret_cast(min_data)); - filter->mutable_max_val()->set_intval(*reinterpret_cast(max_data)); - return; - } - case TYPE_IPV6: { - filter->mutable_min_val()->set_stringval( - LargeIntValue::to_string(*reinterpret_cast(min_data))); - filter->mutable_max_val()->set_stringval( - LargeIntValue::to_string(*reinterpret_cast(max_data))); - return; - } - default: { - throw Exception(ErrorCode::INTERNAL_ERROR, - "runtime filter meet invalid PrimitiveType type {}", - int(_wrapper->column_type())); - } - } + _wrapper->_context->minmax_func->to_pb(filter); } RuntimeFilterType IRuntimeFilter::get_real_type() { diff --git a/be/src/exprs/runtime_filter_convertor.h b/be/src/exprs/runtime_filter_convertor.h new file mode 100644 index 000000000000000..82df75e4abf329d --- /dev/null +++ b/be/src/exprs/runtime_filter_convertor.h @@ -0,0 +1,76 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include "runtime/large_int_value.h" +#include "vec/common/string_ref.h" +#include "vec/core/wide_integer.h" + +namespace doris { + +template +auto get_convertor() { + if constexpr (std::is_same_v) { + return [](PColumnValue* value, const T& data) { value->set_boolval(data); }; + } else if constexpr (std::is_same_v || std::is_same_v || + std::is_same_v || std::is_same_v || + std::is_same_v) { + return [](PColumnValue* value, const T& data) { value->set_intval(data); }; + } else if constexpr (std::is_same_v || std::is_same_v) { + return [](PColumnValue* value, const T& data) { value->set_longval(data); }; + } else if constexpr (std::is_same_v || std::is_same_v) { + return [](PColumnValue* value, const T& data) { value->set_doubleval(data); }; + } else if constexpr (std::is_same_v || std::is_same_v || + std::is_same_v) { + return [](PColumnValue* value, const T& data) { + value->set_stringval(LargeIntValue::to_string(data)); + }; + } else if constexpr (std::is_same_v) { + return [](PColumnValue* value, const T& data) { + value->set_stringval(wide::to_string(wide::Int256(data))); + }; + } else if constexpr (std::is_same_v) { + return [](PColumnValue* value, const T& data) { value->set_stringval(data); }; + } else if constexpr (std::is_same_v || + std::is_same_v || + std::is_same_v) { + return [](PColumnValue* value, const T& data) { value->set_stringval(data.to_string()); }; + } else if constexpr (std::is_same_v) { + return [](PColumnValue* value, const T& data) { + char convert_buffer[30]; + data.to_string(convert_buffer); + value->set_stringval(convert_buffer); + }; + } else if constexpr (std::is_same_v>) { + return [](PColumnValue* value, const T& data) { + value->set_intval(data.to_date_int_val()); + }; + } else if constexpr (std::is_same_v>) { + return [](PColumnValue* value, const T& data) { + value->set_longval(data.to_date_int_val()); + }; + } else { + throw Exception(ErrorCode::INTERNAL_ERROR, + "runtime filter data convertor meet invalid type {}", typeid(T).name()); + return [](PColumnValue* value, const T& data) {}; + } +} + +} // namespace doris diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 7d3f4da935099ee..5c1fa9daa0d8377 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -157,7 +157,8 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu } } SCOPED_TIMER(_publish_runtime_filter_timer); - RETURN_IF_ERROR(_runtime_filter_slots->publish(state, !_should_build_hash_table)); + RETURN_IF_ERROR_OR_CATCH_EXCEPTION( + _runtime_filter_slots->publish(state, !_should_build_hash_table)); return Base::close(state, exec_status); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index e15133246190f9b..2c09ee73f57168b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -2433,6 +2433,8 @@ public void initFuzzyModeVariables() { this.runtimeFilterType = 1 << randomInt; this.enableParallelScan = Config.pull_request_id % 2 == 0 ? randomInt % 2 == 0 : randomInt % 1 == 0; + this.enableRuntimeFilterPrune = (randomInt % 2) == 0; + switch (randomInt) { case 0: this.parallelScanMaxScannersCount = 32; diff --git a/regression-test/data/datatype_p0/ip/ip_rf/test_ip_rf.out b/regression-test/data/datatype_p0/ip/ip_rf/test_ip_rf.out index 7e8a59791f636e4..ae05f362144d0a1 100644 --- a/regression-test/data/datatype_p0/ip/ip_rf/test_ip_rf.out +++ b/regression-test/data/datatype_p0/ip/ip_rf/test_ip_rf.out @@ -29,3 +29,33 @@ -- !sql -- 2 +-- !sql -- +2 + +-- !sql -- +2 + +-- !sql -- +2 + +-- !sql -- +2 + +-- !sql -- +2 + +-- !sql -- +2 + +-- !sql -- +2 + +-- !sql -- +2 + +-- !sql -- +2 + +-- !sql -- +2 + diff --git a/regression-test/suites/datatype_p0/ip/ip_rf/test_ip_rf.groovy b/regression-test/suites/datatype_p0/ip/ip_rf/test_ip_rf.groovy index c7f6c30481eb0b9..8ca5e4a42c54860 100644 --- a/regression-test/suites/datatype_p0/ip/ip_rf/test_ip_rf.groovy +++ b/regression-test/suites/datatype_p0/ip/ip_rf/test_ip_rf.groovy @@ -16,6 +16,8 @@ // specific language governing permissions and limitations // under the License. suite("test_ip_rf") { + sql "set enable_runtime_filter_prune=false;" + sql """ DROP TABLE IF EXISTS ip_test """ sql """ DROP TABLE IF EXISTS ip_test2 """ sql """ @@ -57,17 +59,26 @@ suite("test_ip_rf") { sql "set runtime_filter_type=0;" qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v4=b.ip_v4;" qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v6=b.ip_v6;" + qt_sql "select count(*) from ip_test a, ip_test2 b, ip_test c where a.ip_v4=b.ip_v4 and c.ip_v4=b.ip_v4;" + qt_sql "select count(*) from ip_test a, ip_test2 b, ip_test c where a.ip_v6=b.ip_v6 and c.ip_v6=b.ip_v6;" sql "set runtime_filter_type=1;" qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v4=b.ip_v4;" qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v6=b.ip_v6;" + qt_sql "select count(*) from ip_test a, ip_test2 b, ip_test c where a.ip_v4=b.ip_v4 and c.ip_v4=b.ip_v4;" + qt_sql "select count(*) from ip_test a, ip_test2 b, ip_test c where a.ip_v6=b.ip_v6 and c.ip_v6=b.ip_v6;" sql "set runtime_filter_type=2;" qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v4=b.ip_v4;" qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v6=b.ip_v6;" + qt_sql "select count(*) from ip_test a, ip_test2 b, ip_test c where a.ip_v4=b.ip_v4 and c.ip_v4=b.ip_v4;" + qt_sql "select count(*) from ip_test a, ip_test2 b, ip_test c where a.ip_v6=b.ip_v6 and c.ip_v6=b.ip_v6;" sql "set runtime_filter_type=4;" qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v4=b.ip_v4;" qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v6=b.ip_v6;" + qt_sql "select count(*) from ip_test a, ip_test2 b, ip_test c where a.ip_v4=b.ip_v4 and c.ip_v4=b.ip_v4;" + qt_sql "select count(*) from ip_test a, ip_test2 b, ip_test c where a.ip_v6=b.ip_v6 and c.ip_v6=b.ip_v6;" sql "set runtime_filter_type=8;" qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v4=b.ip_v4;" qt_sql "select count(*) from ip_test a, ip_test2 b where a.ip_v6=b.ip_v6;" - + qt_sql "select count(*) from ip_test a, ip_test2 b, ip_test c where a.ip_v4=b.ip_v4 and c.ip_v4=b.ip_v4;" + qt_sql "select count(*) from ip_test a, ip_test2 b, ip_test c where a.ip_v6=b.ip_v6 and c.ip_v6=b.ip_v6;" } diff --git a/regression-test/suites/nereids_p0/join/test_join_on.groovy b/regression-test/suites/nereids_p0/join/test_join_on.groovy index 2415a3496cf7183..ebd43a544cf9eb0 100644 --- a/regression-test/suites/nereids_p0/join/test_join_on.groovy +++ b/regression-test/suites/nereids_p0/join/test_join_on.groovy @@ -39,16 +39,16 @@ suite("test_join_on", "nereids_p0") { qt_sql """ select * from join_on order by k1; """ test { sql """ select * from join_on as j1 inner join join_on as j2 on j1.d_array = j2.d_array; """ - exception "meet invalid type, type=Array(Nullable(Int32))" + exception "errCode = 2" } test { sql """ select * from join_on as j1 inner join join_on as j2 on j1.hll_col = j2.hll_col; """ - exception "data type HLL could not used in ComparisonPredicate" + exception "errCode = 2" } test { sql """ select * from join_on as j1 inner join join_on as j2 on j1.k3 = j2.k3; """ - exception "data type BITMAP could not used in ComparisonPredicate" + exception "errCode = 2" } test { From 447a3e9eb64ca44f31d0037ed05c5bdf80b2d4f2 Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Tue, 26 Nov 2024 11:34:37 +0800 Subject: [PATCH 092/110] [Fix](Job)Fix some issues in the Insert job. (#44543) ### What problem does this PR solve? - The job does not account for tasks in the Canceled state. - When a job is canceled, its status is marked as FAILED, and a NullPointerException (NPE) occurs because resources have already been released. ``` java.lang.NullPointerException: Cannot invoke "org.apache.doris.qe.ConnectContext.getState()" because "this.ctx" is null at org.apache.doris.job.extensions.insert.InsertTask.run(InsertTask.java:200) ~[classes/:?] at org.apache.doris.job.task.AbstractTask.runTask(AbstractTask.java:167) ~[classes/:?] at org.apache.doris.job.executor.DefaultTaskExecutorHandler.onEvent(DefaultTaskExecutorHandler.java:50) ~[classes/:?] at org.apache.doris.job.executor.DefaultTaskExecutorHandler.onEvent(DefaultTaskExecutorHandler.java:33) ~[classes/:?] at com.lmax.disruptor.WorkProcessor.run(WorkProcessor.java:143) ~[disruptor-3.4.4.jar:?] at java.lang.Thread.run(Thread.java:833) ~[?:?]15e8716a7ab9"> ``` - The RESUME job does not immediately schedule the job. --- .../java/org/apache/doris/job/base/AbstractJob.java | 8 +++++--- .../doris/job/extensions/insert/InsertTask.java | 3 +++ .../java/org/apache/doris/job/manager/JobManager.java | 3 +++ .../java/org/apache/doris/job/task/AbstractTask.java | 3 +++ .../suites/job_p0/test_base_insert_job.groovy | 11 ++++++++++- 5 files changed, 24 insertions(+), 4 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java index 62ac0c4d59d743e..906b86494fb7480 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java @@ -155,6 +155,7 @@ public void cancelAllTasks() throws JobException { } for (T task : runningTasks) { task.cancel(); + canceledTaskCount.incrementAndGet(); } runningTasks = new CopyOnWriteArrayList<>(); logUpdateOperation(); @@ -185,6 +186,7 @@ public void cancelTaskById(long taskId) throws JobException { runningTasks.stream().filter(task -> task.getTaskId().equals(taskId)).findFirst() .orElseThrow(() -> new JobException("Not found task id: " + taskId)).cancel(); runningTasks.removeIf(task -> task.getTaskId().equals(taskId)); + canceledTaskCount.incrementAndGet(); if (jobConfig.getExecuteType().equals(JobExecuteType.ONE_TIME)) { updateJobStatus(JobStatus.FINISHED); } @@ -418,13 +420,13 @@ public TRow getTvfInfo() { /** * Generates a common error message when the execution queue is full. * - * @param taskId The ID of the task. - * @param queueConfigName The name of the queue configuration. + * @param taskId The ID of the task. + * @param queueConfigName The name of the queue configuration. * @param executeThreadConfigName The name of the execution thread configuration. * @return A formatted error message. */ protected String commonFormatMsgWhenExecuteQueueFull(Long taskId, String queueConfigName, - String executeThreadConfigName) { + String executeThreadConfigName) { return String.format("Dispatch task failed, jobId: %d, jobName: %s, taskId: %d, the queue size is full, " + "you can increase the queue size by setting the property " + "%s in the fe.conf file or increase the value of " diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java index c997ebcd30e7757..23a367d5d6e1a1b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java @@ -209,6 +209,9 @@ public void run() throws JobException { @Override public void onFail() throws JobException { + if (isCanceled.get()) { + return; + } isFinished.set(true); super.onFail(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java index 39646bab18f7850..47a3a0c5c19d5cf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java @@ -189,6 +189,9 @@ private void dropJob(T dropJob, String jobName) throws JobException { public void alterJobStatus(Long jobId, JobStatus status) throws JobException { checkJobExist(jobId); jobMap.get(jobId).updateJobStatus(status); + if (status.equals(JobStatus.RUNNING)) { + jobScheduler.scheduleOneJob(jobMap.get(jobId)); + } jobMap.get(jobId).logUpdateOperation(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java index f78446aaf85cbf8..8a230c0bd385f77 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java @@ -167,6 +167,9 @@ public void runTask() throws JobException { run(); onSuccess(); } catch (Exception e) { + if (TaskStatus.CANCELED.equals(status)) { + return; + } this.errMsg = e.getMessage(); onFail(); log.warn("execute task error, job id is {}, task id is {}", jobId, taskId, e); diff --git a/regression-test/suites/job_p0/test_base_insert_job.groovy b/regression-test/suites/job_p0/test_base_insert_job.groovy index 19f4422d64fb01d..8a0bb34ca43fd59 100644 --- a/regression-test/suites/job_p0/test_base_insert_job.groovy +++ b/regression-test/suites/job_p0/test_base_insert_job.groovy @@ -219,9 +219,11 @@ suite("test_base_insert_job") { RESUME JOB where jobname = '${jobName}' """ println(tasks.size()) + // test resume job success Awaitility.await("resume-job-test").atMost(60, SECONDS).until({ def afterResumeTasks = sql """ select status from tasks("type"="insert") where JobName= '${jobName}' """ println "resume tasks :" + afterResumeTasks + //resume tasks size should be greater than before pause afterResumeTasks.size() > tasks.size() }) @@ -247,7 +249,6 @@ suite("test_base_insert_job") { CREATE JOB ${jobName} ON SCHEDULE at '2023-11-13 14:18:07' comment 'test' DO insert into ${tableName} (timestamp, type, user_id) values ('2023-03-18','1','12213'); """ } catch (Exception e) { - println e.getMessage() assert e.getMessage().contains("startTimeMs must be greater than current time") } // assert end time less than start time @@ -281,6 +282,14 @@ suite("test_base_insert_job") { } catch (Exception e) { assert e.getMessage().contains("Invalid interval time unit: years") } + // assert interval time unit is -1 + try { + sql """ + CREATE JOB test_error_starts ON SCHEDULE every -1 second comment 'test' DO insert into ${tableName} (timestamp, type, user_id) values ('2023-03-18','1','12213'); + """ + } catch (Exception e) { + assert e.getMessage().contains("expecting INTEGER_VALUE") + } // test keyword as job name sql """ From 25dcd85865a695f53b39b0e3dd9b7571525ecfe6 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Tue, 26 Nov 2024 11:51:28 +0800 Subject: [PATCH 093/110] [enhance](catalog)Unified external partition prune interface (#44567) ### What problem does this PR solve? 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 ### Release note [opt](planner) Unified external partition prune interface --- .../doris/datasource/ExternalTable.java | 53 +++++++++++++++++++ .../datasource/hive/HMSExternalTable.java | 23 ++++++-- .../nereids/rules/analysis/BindRelation.java | 1 - .../rules/rewrite/PruneFileScanPartition.java | 18 +++---- .../trees/plans/logical/LogicalFileScan.java | 11 ++-- 5 files changed, 82 insertions(+), 24 deletions(-) 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 1eadb46fe82eedd..3aee5550acf6466 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 @@ -20,6 +20,7 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.PartitionItem; import org.apache.doris.catalog.TableAttributes; import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.TableIndexes; @@ -30,6 +31,7 @@ import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.common.util.Util; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan.SelectedPartitions; import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.statistics.AnalysisInfo; @@ -41,6 +43,7 @@ import com.google.common.collect.Sets; import com.google.gson.annotations.SerializedName; import lombok.Getter; +import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.NotImplementedException; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -48,9 +51,11 @@ import java.io.DataInput; import java.io.DataOutput; 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.Set; /** @@ -364,4 +369,52 @@ protected Optional getSchemaCacheValue() { public TableIndexes getTableIndexes() { return new TableIndexes(); } + + /** + * Retrieve all partitions and initialize SelectedPartitions + * + * @param snapshotId if not support mvcc, ignore this + * @return + */ + public SelectedPartitions initSelectedPartitions(OptionalLong snapshotId) { + if (!supportPartitionPruned()) { + return SelectedPartitions.NOT_PRUNED; + } + if (CollectionUtils.isEmpty(this.getPartitionColumns(snapshotId))) { + return SelectedPartitions.NOT_PRUNED; + } + Map nameToPartitionItems = getNameToPartitionItems(snapshotId); + return new SelectedPartitions(nameToPartitionItems.size(), nameToPartitionItems, false); + } + + /** + * get partition map + * If partition related operations are supported, this method needs to be implemented in the subclass + * + * @param snapshotId if not support mvcc, ignore this + * @return partitionName ==> PartitionItem + */ + public Map getNameToPartitionItems(OptionalLong snapshotId) { + return Collections.emptyMap(); + } + + /** + * get partition column list + * If partition related operations are supported, this method needs to be implemented in the subclass + * + * @param snapshotId if not support mvcc, ignore this + * @return + */ + public List getPartitionColumns(OptionalLong snapshotId) { + return Collections.emptyList(); + } + + /** + * Does it support partition cpruned, If so, this method needs to be overridden in subclasses + * + * @return + */ + public boolean supportPartitionPruned() { + return false; + } } 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 689b5a9b59a4d44..1c30fa24cfb51e7 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 @@ -40,7 +40,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; @@ -297,11 +296,25 @@ public List getPartitionColumns() { .orElse(Collections.emptyList()); } - public SelectedPartitions getAllPartitions() { + @Override + public List getPartitionColumns(OptionalLong snapshotId) { + return getPartitionColumns(); + } + + @Override + public boolean supportPartitionPruned() { + return getDlaType() == DLAType.HIVE; + } + + @Override + public Map getNameToPartitionItems(OptionalLong snapshotId) { + return getNameToPartitionItems(); + } + + public Map getNameToPartitionItems() { if (CollectionUtils.isEmpty(this.getPartitionColumns())) { - return SelectedPartitions.NOT_PRUNED; + return Collections.emptyMap(); } - HiveMetaStoreCache cache = Env.getCurrentEnv().getExtMetaCacheMgr() .getMetaStoreCache((HMSExternalCatalog) this.getCatalog()); List partitionColumnTypes = this.getPartitionColumnTypes(); @@ -314,7 +327,7 @@ public SelectedPartitions getAllPartitions() { for (Entry entry : idToPartitionItem.entrySet()) { nameToPartitionItem.put(idToName.get(entry.getKey()), entry.getValue()); } - return new SelectedPartitions(idToPartitionItem.size(), nameToPartitionItem, false); + return nameToPartitionItem; } public boolean isHiveTransactionalTable() { 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 9699ca243cf9896..cba3afca6f08d3b 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 @@ -435,7 +435,6 @@ private LogicalPlan getLogicalPlan(TableIf table, UnboundRelation unboundRelatio } else { return new LogicalFileScan(unboundRelation.getRelationId(), (HMSExternalTable) table, qualifierWithoutTableName, - ((HMSExternalTable) table).getAllPartitions(), unboundRelation.getTableSample(), unboundRelation.getTableSnapshot()); } 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 9bec65708222408..d50219383072df0 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 @@ -19,8 +19,6 @@ import org.apache.doris.catalog.PartitionItem; import org.apache.doris.datasource.ExternalTable; -import org.apache.doris.datasource.hive.HMSExternalTable; -import org.apache.doris.datasource.hive.HMSExternalTable.DLAType; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; @@ -38,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; @@ -60,10 +59,8 @@ public Rule build() { ExternalTable tbl = scan.getTable(); SelectedPartitions selectedPartitions; - // TODO(cmy): support other external table - if (tbl instanceof HMSExternalTable && ((HMSExternalTable) tbl).getDlaType() == DLAType.HIVE) { - HMSExternalTable hiveTbl = (HMSExternalTable) tbl; - selectedPartitions = pruneHivePartitions(hiveTbl, filter, scan, ctx.cascadesContext); + if (tbl.supportPartitionPruned()) { + selectedPartitions = pruneExternalPartitions(tbl, filter, scan, ctx.cascadesContext); } else { // set isPruned so that it won't go pass the partition prune again selectedPartitions = new SelectedPartitions(0, ImmutableMap.of(), true); @@ -74,10 +71,11 @@ public Rule build() { }).toRule(RuleType.FILE_SCAN_PARTITION_PRUNE); } - private SelectedPartitions pruneHivePartitions(HMSExternalTable hiveTbl, + private SelectedPartitions pruneExternalPartitions(ExternalTable externalTable, LogicalFilter filter, LogicalFileScan scan, CascadesContext ctx) { Map selectedPartitionItems = Maps.newHashMap(); - if (CollectionUtils.isEmpty(hiveTbl.getPartitionColumns())) { + // todo: real snapshotId + if (CollectionUtils.isEmpty(externalTable.getPartitionColumns(OptionalLong.empty()))) { // non partitioned table, return NOT_PRUNED. // non partition table will be handled in HiveScanNode. return SelectedPartitions.NOT_PRUNED; @@ -85,8 +83,8 @@ private SelectedPartitions pruneHivePartitions(HMSExternalTable hiveTbl, Map scanOutput = scan.getOutput() .stream() .collect(Collectors.toMap(slot -> slot.getName().toLowerCase(), Function.identity())); - - List partitionSlots = hiveTbl.getPartitionColumns() + // todo: real snapshotId + List partitionSlots = externalTable.getPartitionColumns(OptionalLong.empty()) .stream() .map(column -> scanOutput.get(column.getName().toLowerCase())) .collect(Collectors.toList()); 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 ab4ef8efa3c5dfd..010c30d915d529b 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 @@ -36,6 +36,7 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.OptionalLong; /** * Logical file scan for external catalog. @@ -59,17 +60,11 @@ protected LogicalFileScan(RelationId id, ExternalTable table, List quali this.tableSnapshot = tableSnapshot; } - public LogicalFileScan(RelationId id, ExternalTable table, List qualifier, - SelectedPartitions selectedPartitions, - Optional tableSample, Optional tableSnapshot) { - this(id, table, qualifier, Optional.empty(), Optional.empty(), - selectedPartitions, tableSample, tableSnapshot); - } - public LogicalFileScan(RelationId id, ExternalTable table, List qualifier, Optional tableSample, Optional tableSnapshot) { + // todo: real snapshotId this(id, table, qualifier, Optional.empty(), Optional.empty(), - SelectedPartitions.NOT_PRUNED, tableSample, tableSnapshot); + table.initSelectedPartitions(OptionalLong.empty()), tableSample, tableSnapshot); } public SelectedPartitions getSelectedPartitions() { From 6ce02086a1775170e2c4d5d525f8f478ce478ffd Mon Sep 17 00:00:00 2001 From: Jerry Hu Date: Tue, 26 Nov 2024 12:34:55 +0800 Subject: [PATCH 094/110] [opt](brpc) Make the timeout of brpc checking configurable (#44481) --- be/src/common/config.cpp | 2 ++ be/src/common/config.h | 2 ++ be/src/runtime/fragment_mgr.cpp | 5 ++++- 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 0cffa30cdca4434..63989a76261bb6d 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -554,6 +554,8 @@ DEFINE_Bool(enable_brpc_builtin_services, "true"); // Enable brpc connection check DEFINE_Bool(enable_brpc_connection_check, "false"); +DEFINE_mInt64(brpc_connection_check_timeout_ms, "10000"); + // The maximum amount of data that can be processed by a stream load DEFINE_mInt64(streaming_load_max_mb, "102400"); // Some data formats, such as JSON, cannot be streamed. diff --git a/be/src/common/config.h b/be/src/common/config.h index caee1f320c103e5..29e55e6406390e7 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1016,6 +1016,8 @@ DECLARE_Bool(enable_brpc_builtin_services); DECLARE_Bool(enable_brpc_connection_check); +DECLARE_mInt64(brpc_connection_check_timeout_ms); + // Max waiting time to wait the "plan fragment start" rpc. // If timeout, the fragment will be cancelled. // This parameter is usually only used when the FE loses connection, diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index 47b007edf5dc9bc..1e72fa756d3dd3c 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -1062,12 +1062,15 @@ void FragmentMgr::_check_brpc_available(const std::shared_ptr(100, config::brpc_connection_check_timeout_ms); + while (true) { PHandShakeRequest request; request.set_hello(message); PHandShakeResponse response; brpc::Controller cntl; - cntl.set_timeout_ms(500 * (failed_count + 1)); + cntl.set_timeout_ms(check_timeout_ms); cntl.set_max_retry(10); brpc_stub->hand_shake(&cntl, &request, &response, nullptr); From 29787a2985aee8e50d687cc82ba7f23bb7a1a5e3 Mon Sep 17 00:00:00 2001 From: shuke Date: Tue, 26 Nov 2024 14:19:47 +0800 Subject: [PATCH 095/110] [regression-test](fix) fix case failed due to global variable (#44564) --- .../schema_change_p0/test_agg_vals_schema_change.groovy | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/regression-test/suites/schema_change_p0/test_agg_vals_schema_change.groovy b/regression-test/suites/schema_change_p0/test_agg_vals_schema_change.groovy index 165fba94b9a63d1..477ea202e4c38ea 100644 --- a/regression-test/suites/schema_change_p0/test_agg_vals_schema_change.groovy +++ b/regression-test/suites/schema_change_p0/test_agg_vals_schema_change.groovy @@ -141,7 +141,7 @@ suite ("test_agg_vals_schema_change") { String tablet_id = tablet[0] backend_id = tablet[2] logger.info("run compaction:" + tablet_id) - (code, out, err) = be_run_cumulative_compaction(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + def (code, out, err) = be_run_cumulative_compaction(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) //assertEquals(code, 0) } @@ -153,7 +153,7 @@ suite ("test_agg_vals_schema_change") { Thread.sleep(100) String tablet_id = tablet[0] backend_id = tablet[2] - (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + def (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) assertEquals(code, 0) def compactionStatus = parseJson(out.trim()) From 0bc3ea7728b81a253dc5a6122bab9f332856d21d Mon Sep 17 00:00:00 2001 From: morrySnow Date: Tue, 26 Nov 2024 14:32:33 +0800 Subject: [PATCH 096/110] [chore](Nereids) update some exception type and message (#44484) --- .../pattern/GroupExpressionMatching.java | 2 +- .../rules/FoldConstantRuleOnBE.java | 4 ++-- .../rules/RangePartitionValueIterator.java | 2 +- .../doris/nereids/stats/StatsCalculator.java | 7 +++--- .../commands/ShowTabletsBelongCommand.java | 3 ++- .../plans/commands/info/DefaultValue.java | 3 ++- .../commands/info/PartitionTableInfo.java | 22 +++++++++---------- .../worker/job/UnassignedJobBuilder.java | 3 ++- 8 files changed, 23 insertions(+), 23 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/GroupExpressionMatching.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/GroupExpressionMatching.java index e281e74a339cc76..292e3e5ccf8e7f0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/GroupExpressionMatching.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/GroupExpressionMatching.java @@ -200,7 +200,7 @@ public boolean hasNext() { @Override public Plan next() { if (!hasNext()) { - throw new NoSuchElementException(); + throw new NoSuchElementException("GroupExpressionIterator is empty"); } return results.get(resultIndex++); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnBE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnBE.java index 64f7c2784f77796..70e63b050a84029 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnBE.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnBE.java @@ -25,7 +25,6 @@ import org.apache.doris.common.Config; import org.apache.doris.common.IdGenerator; import org.apache.doris.common.Pair; -import org.apache.doris.common.UserException; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.nereids.glue.translator.ExpressionTranslator; @@ -272,7 +271,8 @@ private static Map evalOnBE(Map> try { List backendIds = Env.getCurrentSystemInfo().getAllBackendByCurrentCluster(true); if (backendIds.isEmpty()) { - throw new UserException("No alive backends"); + LOG.warn("no available backend ids for folding constant on BE"); + return Collections.emptyMap(); } Collections.shuffle(backendIds); Backend be = Env.getCurrentSystemInfo().getBackend(backendIds.get(0)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/RangePartitionValueIterator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/RangePartitionValueIterator.java index 79ee33d1ebb815c..f9ed495f46dd8fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/RangePartitionValueIterator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/RangePartitionValueIterator.java @@ -57,7 +57,7 @@ public L next() { current = doGetNext(current); return toLiteral.apply(value); } - throw new NoSuchElementException(); + throw new NoSuchElementException("RangePartitionValueIterator is empty"); } protected abstract C doGetNext(C current); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java index 499f202b05b6200..f549b21621758dd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java @@ -29,7 +29,6 @@ import org.apache.doris.catalog.PartitionType; import org.apache.doris.catalog.RangePartitionItem; import org.apache.doris.catalog.TableIf; -import org.apache.doris.common.AnalysisException; import org.apache.doris.common.FeConstants; import org.apache.doris.common.Pair; import org.apache.doris.nereids.CascadesContext; @@ -600,7 +599,7 @@ private ColumnStatistic updateMinMaxForPartitionKey(OlapTable olapTable, return cache; } - private double convertLegacyLiteralToDouble(LiteralExpr literal) throws AnalysisException { + private double convertLegacyLiteralToDouble(LiteralExpr literal) throws org.apache.doris.common.AnalysisException { return StatisticsUtil.convertToDouble(literal.getType(), literal.getStringValue()); } @@ -641,7 +640,7 @@ private ColumnStatistic updateMinMaxForListPartitionKey(OlapTable olapTable, if (minExpr != null) { cache = updateMinMax(cache, minValue, minExpr, maxValue, maxExpr); } - } catch (AnalysisException e) { + } catch (org.apache.doris.common.AnalysisException e) { LOG.debug(e.getMessage()); } } @@ -691,7 +690,7 @@ private ColumnStatistic updateMinMaxForTheFirstRangePartitionKey(OlapTable olapT if (minExpr != null) { cache = updateMinMax(cache, minValue, minExpr, maxValue, maxExpr); } - } catch (AnalysisException e) { + } catch (org.apache.doris.common.AnalysisException e) { LOG.debug(e.getMessage()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTabletsBelongCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTabletsBelongCommand.java index 2243e60868f988f..399b63e6037ecf3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTabletsBelongCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTabletsBelongCommand.java @@ -31,6 +31,7 @@ import org.apache.doris.common.Pair; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.qe.ConnectContext; @@ -87,7 +88,7 @@ public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exc PrivPredicate.ADMIN.getPrivs().toString()); } if (tabletIds == null || tabletIds.isEmpty()) { - throw new Exception("Please supply at least one tablet id"); + throw new AnalysisException("Please supply at least one tablet id"); } // main logic. diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DefaultValue.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DefaultValue.java index e3f388d272deebc..465f0f9f32bd998 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DefaultValue.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DefaultValue.java @@ -19,6 +19,7 @@ import org.apache.doris.analysis.DefaultValueExprDef; import org.apache.doris.catalog.ScalarType; +import org.apache.doris.nereids.exceptions.AnalysisException; /** * default value of a column. @@ -77,7 +78,7 @@ public DefaultValue(String value, String exprName, Long precision) { */ public static DefaultValue currentTimeStampDefaultValueWithPrecision(Long precision) { if (precision > ScalarType.MAX_DATETIMEV2_SCALE || precision < 0) { - throw new IllegalArgumentException("column's default value current_timestamp" + throw new AnalysisException("column's default value current_timestamp" + " precision must be between 0 and 6"); } if (precision == 0) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/PartitionTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/PartitionTableInfo.java index 71aab1a04cbe78b..7d85d24f6cb74a6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/PartitionTableInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/PartitionTableInfo.java @@ -28,7 +28,6 @@ import org.apache.doris.analysis.StringLiteral; import org.apache.doris.catalog.AggregateType; import org.apache.doris.catalog.PartitionType; -import org.apache.doris.common.DdlException; import org.apache.doris.nereids.analyzer.UnboundFunction; import org.apache.doris.nereids.analyzer.UnboundSlot; import org.apache.doris.nereids.exceptions.AnalysisException; @@ -261,16 +260,15 @@ public PartitionDesc convertToPartitionDesc(boolean isExternal) { partitionDescs.size(), createTablePartitionMaxNum)); } - try { - ArrayList exprs = convertToLegacyAutoPartitionExprs(partitionList); + ArrayList exprs = convertToLegacyAutoPartitionExprs(partitionList); - // only auto partition support partition expr - if (!isAutoPartition) { - if (exprs.stream().anyMatch(expr -> expr instanceof FunctionCallExpr)) { - throw new DdlException("Non-auto partition table not support partition expr!"); - } + // only auto partition support partition expr + if (!isAutoPartition) { + if (exprs.stream().anyMatch(expr -> expr instanceof FunctionCallExpr)) { + throw new AnalysisException("Non-auto partition table not support partition expr!"); } - + } + try { // here we have already extracted identifierPartitionColumns if (partitionType.equals(PartitionType.RANGE.name())) { if (isAutoPartition) { @@ -293,7 +291,7 @@ public PartitionDesc convertToPartitionDesc(boolean isExternal) { } private static ArrayList convertToLegacyAutoPartitionExprs(List expressions) { - return new ArrayList<>(expressions.stream().map(expression -> { + return expressions.stream().map(expression -> { if (expression instanceof UnboundSlot) { return new SlotRef(null, ((UnboundSlot) expression).getName()); } else if (expression instanceof UnboundFunction) { @@ -303,9 +301,9 @@ private static ArrayList convertToLegacyAutoPartitionExprs(List convertToLegacyArguments(List children) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJobBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJobBuilder.java index 64ff43da8c837a6..247315c3011378e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJobBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJobBuilder.java @@ -130,7 +130,8 @@ private UnassignedJob buildLeafOrScanJob( return unassignedJob; } - throw new IllegalStateException("Unsupported build UnassignedJob for fragment: " + throw new IllegalStateException("Cannot generate unassignedJob for fragment" + + " has both OlapScanNode and Other ScanNode: " + planFragment.getExplainString(TExplainLevel.VERBOSE)); } From 378b2f21e22b11e8b912b077f2032e63c1ca0c81 Mon Sep 17 00:00:00 2001 From: Mryange Date: Tue, 26 Nov 2024 15:41:38 +0800 Subject: [PATCH 097/110] [opt](exec)lazy deserialize pblock in VDataStreamRecvr::SenderQueue (#44378) ### What problem does this PR solve? Previously, for a `pblock` (serialized block), the block would be deserialized immediately after receiving the RPC request and then placed into the `data_queue`. This approach caused significant time consumption during RPC processing due to the deserialization process, impacting overall performance. The new approach defers deserialization until `getBlock` is called. This has the following advantages: 1. Reduces time spent during the RPC handling phase. 2. Memory allocation for deserialization happens within the execution thread, improving cache locality and reducing contention on memory resources. ### Check List (For Author) - Test - [ ] Regression test - [ ] Unit Test - [ ] Manual test (add detailed scripts or steps below) - [x] No need to test or manual test. Explain why: - [x] This is a refactor/code format and no logic has been changed. - [ ] Previous test can cover this change. - [ ] No code files have been changed. - [ ] Other reason - Behavior changed: - [x] No. - [ ] Yes. - Does this need documentation? - [x] No. - [ ] Yes. ### Check List (For Reviewer who merge this PR) - [ ] Confirm the release note - [ ] Confirm test cases - [ ] Confirm document - [ ] Add branch pick label --- be/src/vec/runtime/vdata_stream_mgr.cpp | 11 +++- be/src/vec/runtime/vdata_stream_recvr.cpp | 68 ++++++++++------------- be/src/vec/runtime/vdata_stream_recvr.h | 49 +++++++++++++--- 3 files changed, 76 insertions(+), 52 deletions(-) diff --git a/be/src/vec/runtime/vdata_stream_mgr.cpp b/be/src/vec/runtime/vdata_stream_mgr.cpp index 382a6d0e6e3eced..7dad3d2c86705e7 100644 --- a/be/src/vec/runtime/vdata_stream_mgr.cpp +++ b/be/src/vec/runtime/vdata_stream_mgr.cpp @@ -18,10 +18,12 @@ #include "vec/runtime/vdata_stream_mgr.h" #include +#include #include #include #include +#include #include #include #include @@ -141,9 +143,12 @@ Status VDataStreamMgr::transmit_block(const PTransmitDataParams* request, bool eos = request->eos(); if (request->has_block()) { - RETURN_IF_ERROR(recvr->add_block( - request->block(), request->sender_id(), request->be_number(), request->packet_seq(), - eos ? nullptr : done, wait_for_worker, cpu_time_stop_watch.elapsed_time())); + std::unique_ptr pblock_ptr { + const_cast(request)->release_block()}; + RETURN_IF_ERROR(recvr->add_block(std::move(pblock_ptr), request->sender_id(), + request->be_number(), request->packet_seq(), + eos ? nullptr : done, wait_for_worker, + cpu_time_stop_watch.elapsed_time())); } if (eos) { diff --git a/be/src/vec/runtime/vdata_stream_recvr.cpp b/be/src/vec/runtime/vdata_stream_recvr.cpp index b48b9f780b8754e..81e4e1cd5f037e4 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.cpp +++ b/be/src/vec/runtime/vdata_stream_recvr.cpp @@ -69,7 +69,6 @@ VDataStreamRecvr::SenderQueue::~SenderQueue() { } Status VDataStreamRecvr::SenderQueue::get_batch(Block* block, bool* eos) { - std::lock_guard l(_lock); // protect _block_queue #ifndef NDEBUG if (!_is_cancelled && _block_queue.empty() && _num_remaining_senders > 0) { throw doris::Exception(ErrorCode::INTERNAL_ERROR, @@ -79,25 +78,33 @@ Status VDataStreamRecvr::SenderQueue::get_batch(Block* block, bool* eos) { _debug_string_info()); } #endif - return _inner_get_batch_without_lock(block, eos); -} + BlockItem block_item; + { + std::lock_guard l(_lock); + //check and get block_item from data_queue + if (_is_cancelled) { + RETURN_IF_ERROR(_cancel_status); + return Status::Cancelled("Cancelled"); + } -Status VDataStreamRecvr::SenderQueue::_inner_get_batch_without_lock(Block* block, bool* eos) { - if (_is_cancelled) { - RETURN_IF_ERROR(_cancel_status); - return Status::Cancelled("Cancelled"); - } + if (_block_queue.empty()) { + DCHECK_EQ(_num_remaining_senders, 0); + *eos = true; + return Status::OK(); + } - if (_block_queue.empty()) { - DCHECK_EQ(_num_remaining_senders, 0); - *eos = true; - return Status::OK(); + DCHECK(!_block_queue.empty()); + block_item = std::move(_block_queue.front()); + _block_queue.pop_front(); } - - DCHECK(!_block_queue.empty()); - auto [next_block, block_byte_size] = std::move(_block_queue.front()); - _block_queue.pop_front(); + BlockUPtr next_block; + RETURN_IF_ERROR(block_item.get_block(next_block)); + size_t block_byte_size = block_item.block_byte_size(); + COUNTER_UPDATE(_recvr->_deserialize_row_batch_timer, block_item.deserialize_time()); + COUNTER_UPDATE(_recvr->_decompress_timer, block->get_decompress_time()); + COUNTER_UPDATE(_recvr->_decompress_bytes, block->get_decompressed_bytes()); _recvr->_parent->memory_used_counter()->update(-(int64_t)block_byte_size); + std::lock_guard l(_lock); sub_blocks_memory_usage(block_byte_size); _record_debug_info(); if (_block_queue.empty() && _source_dependency) { @@ -133,7 +140,7 @@ void VDataStreamRecvr::SenderQueue::try_set_dep_ready_without_lock() { } } -Status VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_number, +Status VDataStreamRecvr::SenderQueue::add_block(std::unique_ptr pblock, int be_number, int64_t packet_seq, ::google::protobuf::Closure** done, const int64_t wait_for_worker, @@ -163,30 +170,12 @@ Status VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_num } } - BlockUPtr block = nullptr; - int64_t deserialize_time = 0; - { - SCOPED_RAW_TIMER(&deserialize_time); - block = Block::create_unique(); - RETURN_IF_ERROR_OR_CATCH_EXCEPTION(block->deserialize(pblock)); - } - - const auto rows = block->rows(); - if (rows == 0) { - return Status::OK(); - } - auto block_byte_size = block->allocated_bytes(); - VLOG_ROW << "added #rows=" << rows << " batch_size=" << block_byte_size << "\n"; - std::lock_guard l(_lock); if (_is_cancelled) { return Status::OK(); } - COUNTER_UPDATE(_recvr->_deserialize_row_batch_timer, deserialize_time); - COUNTER_UPDATE(_recvr->_decompress_timer, block->get_decompress_time()); - COUNTER_UPDATE(_recvr->_decompress_bytes, block->get_decompressed_bytes()); - COUNTER_UPDATE(_recvr->_rows_produced_counter, rows); + const auto block_byte_size = pblock->ByteSizeLong(); COUNTER_UPDATE(_recvr->_blocks_produced_counter, 1); if (_recvr->_max_wait_worker_time->value() < wait_for_worker) { _recvr->_max_wait_worker_time->set(wait_for_worker); @@ -196,7 +185,7 @@ Status VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_num _recvr->_max_find_recvr_time->set((int64_t)time_to_find_recvr); } - _block_queue.emplace_back(std::move(block), block_byte_size); + _block_queue.emplace_back(std::move(pblock), block_byte_size); COUNTER_UPDATE(_recvr->_remote_bytes_received_counter, block_byte_size); _record_debug_info(); try_set_dep_ready_without_lock(); @@ -370,7 +359,6 @@ VDataStreamRecvr::VDataStreamRecvr(VDataStreamMgr* stream_mgr, pipeline::Exchang _first_batch_wait_total_timer = ADD_TIMER(_profile, "FirstBatchArrivalWaitTime"); _decompress_timer = ADD_TIMER(_profile, "DecompressTime"); _decompress_bytes = ADD_COUNTER(_profile, "DecompressBytes", TUnit::BYTES); - _rows_produced_counter = ADD_COUNTER(_profile, "RowsProduced", TUnit::UNIT); _blocks_produced_counter = ADD_COUNTER(_profile, "BlocksProduced", TUnit::UNIT); _max_wait_worker_time = ADD_COUNTER(_profile, "MaxWaitForWorkerTime", TUnit::UNIT); _max_wait_to_process_time = ADD_COUNTER(_profile, "MaxWaitToProcessTime", TUnit::UNIT); @@ -401,13 +389,13 @@ Status VDataStreamRecvr::create_merger(const VExprContextSPtrs& ordering_expr, return Status::OK(); } -Status VDataStreamRecvr::add_block(const PBlock& pblock, int sender_id, int be_number, +Status VDataStreamRecvr::add_block(std::unique_ptr pblock, int sender_id, int be_number, int64_t packet_seq, ::google::protobuf::Closure** done, const int64_t wait_for_worker, const uint64_t time_to_find_recvr) { SCOPED_ATTACH_TASK(_query_thread_context); int use_sender_id = _is_merging ? sender_id : 0; - return _sender_queues[use_sender_id]->add_block(pblock, be_number, packet_seq, done, + return _sender_queues[use_sender_id]->add_block(std::move(pblock), be_number, packet_seq, done, wait_for_worker, time_to_find_recvr); } diff --git a/be/src/vec/runtime/vdata_stream_recvr.h b/be/src/vec/runtime/vdata_stream_recvr.h index 08fb004f3b1a8ea..1639366c8b83d68 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.h +++ b/be/src/vec/runtime/vdata_stream_recvr.h @@ -18,6 +18,7 @@ #pragma once #include +#include #include #include @@ -84,9 +85,9 @@ class VDataStreamRecvr : public HasTaskExecutionCtx { std::vector sender_queues() const { return _sender_queues; } - Status add_block(const PBlock& pblock, int sender_id, int be_number, int64_t packet_seq, - ::google::protobuf::Closure** done, const int64_t wait_for_worker, - const uint64_t time_to_find_recvr); + Status add_block(std::unique_ptr pblock, int sender_id, int be_number, + int64_t packet_seq, ::google::protobuf::Closure** done, + const int64_t wait_for_worker, const uint64_t time_to_find_recvr); void add_block(Block* block, int sender_id, bool use_move); @@ -157,8 +158,6 @@ class VDataStreamRecvr : public HasTaskExecutionCtx { RuntimeProfile::Counter* _decompress_timer = nullptr; RuntimeProfile::Counter* _decompress_bytes = nullptr; - // Number of rows received - RuntimeProfile::Counter* _rows_produced_counter = nullptr; // Number of blocks received RuntimeProfile::Counter* _blocks_produced_counter = nullptr; RuntimeProfile::Counter* _max_wait_worker_time = nullptr; @@ -181,7 +180,7 @@ class VDataStreamRecvr::SenderQueue { Status get_batch(Block* next_block, bool* eos); - Status add_block(const PBlock& pblock, int be_number, int64_t packet_seq, + Status add_block(std::unique_ptr pblock, int be_number, int64_t packet_seq, ::google::protobuf::Closure** done, const int64_t wait_for_worker, const uint64_t time_to_find_recvr); @@ -205,8 +204,6 @@ class VDataStreamRecvr::SenderQueue { protected: friend class pipeline::ExchangeLocalState; - Status _inner_get_batch_without_lock(Block* block, bool* eos); - void try_set_dep_ready_without_lock(); // To record information about several variables in the event of a DCHECK failure. @@ -260,7 +257,41 @@ class VDataStreamRecvr::SenderQueue { Status _cancel_status; int _num_remaining_senders; std::unique_ptr _queue_mem_tracker; - std::list> _block_queue; + + // `BlockItem` is used in `_block_queue` to handle both local and remote exchange blocks. + // For local exchange blocks, `BlockUPtr` is used directly without any modification. + // For remote exchange blocks, the `pblock` is stored in `BlockItem`. + // When `getBlock` is called, the `pblock` is deserialized into a usable block. + struct BlockItem { + Status get_block(BlockUPtr& block) { + if (!_block) { + DCHECK(_pblock); + SCOPED_RAW_TIMER(&_deserialize_time); + _block = Block::create_unique(); + RETURN_IF_ERROR_OR_CATCH_EXCEPTION(_block->deserialize(*_pblock)); + } + block.swap(_block); + _block.reset(); + return Status::OK(); + } + + size_t block_byte_size() const { return _block_byte_size; } + int64_t deserialize_time() const { return _deserialize_time; } + BlockItem() = default; + BlockItem(BlockUPtr&& block, size_t block_byte_size) + : _block(std::move(block)), _block_byte_size(block_byte_size) {} + + BlockItem(std::unique_ptr&& pblock, size_t block_byte_size) + : _block(nullptr), _pblock(std::move(pblock)), _block_byte_size(block_byte_size) {} + + private: + BlockUPtr _block; + std::unique_ptr _pblock; + size_t _block_byte_size = 0; + int64_t _deserialize_time = 0; + }; + + std::list _block_queue; // sender_id std::unordered_set _sender_eos_set; From 3358156cf14531e9a9573bbcc965e608d05cdae7 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Tue, 26 Nov 2024 13:29:42 +0530 Subject: [PATCH 098/110] [Enhancement] (nereids)implement showLoadProfileCommand in nereids (#44440) Issue Number: close #42773 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 8 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/ShowLoadProfileCommand.java | 57 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../show/test_show_commands_nereids.groovy | 2 +- 6 files changed, 73 insertions(+), 2 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowLoadProfileCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 47214c8c271af69..30b924ed878a5c3 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -211,6 +211,7 @@ supportedShowStatement | SHOW DELETE ((FROM | IN) database=multipartIdentifier)? #showDelete | SHOW ALL? GRANTS #showGrants | SHOW GRANTS FOR userIdentify #showGrantsForUser + | SHOW LOAD PROFILE loadIdPath=STRING_LITERAL #showLoadProfile | SHOW VIEW (FROM |IN) tableName=multipartIdentifier ((FROM | IN) database=identifier)? #showView @@ -330,7 +331,6 @@ unsupportedShowStatement ((FROM | IN) database=multipartIdentifier)? #showIndex | SHOW TRANSACTION ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTransaction | SHOW QUERY PROFILE queryIdPath=STRING_LITERAL #showQueryProfile - | SHOW LOAD PROFILE loadIdPath=STRING_LITERAL #showLoadProfile | SHOW CACHE HOTSPOT tablePath=STRING_LITERAL #showCacheHotSpot | SHOW ENCRYPTKEYS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showEncryptKeys | SHOW SYNC JOB ((FROM | IN) database=multipartIdentifier)? #showSyncJob diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 5d31b284e15562e..b77c141e2fd6959 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -216,6 +216,7 @@ import org.apache.doris.nereids.DorisParser.ShowGrantsContext; import org.apache.doris.nereids.DorisParser.ShowGrantsForUserContext; import org.apache.doris.nereids.DorisParser.ShowLastInsertContext; +import org.apache.doris.nereids.DorisParser.ShowLoadProfileContext; import org.apache.doris.nereids.DorisParser.ShowPartitionIdContext; import org.apache.doris.nereids.DorisParser.ShowPluginsContext; import org.apache.doris.nereids.DorisParser.ShowPrivilegesContext; @@ -483,6 +484,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowFrontendsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowGrantsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowLastInsertCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowLoadProfileCommand; import org.apache.doris.nereids.trees.plans.commands.ShowPartitionIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowPluginsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowPrivilegesCommand; @@ -4136,10 +4138,16 @@ public Object visitRefreshCatalog(RefreshCatalogContext ctx) { throw new AnalysisException("catalog name can not be null"); } + @Override public LogicalPlan visitShowLastInsert(ShowLastInsertContext ctx) { return new ShowLastInsertCommand(); } + @Override + public LogicalPlan visitShowLoadProfile(ShowLoadProfileContext ctx) { + return new ShowLoadProfileCommand(ctx.loadIdPath.getText()); + } + @Override public LogicalPlan visitShowGrants(ShowGrantsContext ctx) { boolean all = (ctx.ALL() != null) ? true : false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index f36af26e89273e3..e649a905b4a0c78 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -196,6 +196,7 @@ public enum PlanType { SHOW_FRONTENDS_COMMAND, SHOW_GRANTS_COMMAND, SHOW_LAST_INSERT_COMMAND, + SHOW_LOAD_PROFILE_COMMAND, SHOW_PARTITIONID_COMMAND, SHOW_PROC_COMMAND, SHOW_PLUGINS_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowLoadProfileCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowLoadProfileCommand.java new file mode 100644 index 000000000000000..5320671ad7794d5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowLoadProfileCommand.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.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.Config; +import org.apache.doris.common.UserException; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.StmtExecutor; + +/** + * show load profile command + */ +public class ShowLoadProfileCommand extends ShowCommand { + private String loadIdPath; + + /** + * constructor + */ + public ShowLoadProfileCommand(String path) { + super(PlanType.SHOW_LOAD_PROFILE_COMMAND); + this.loadIdPath = path; + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + String selfHost = Env.getCurrentEnv().getSelfNode().getHost(); + int httpPort = Config.http_port; + String terminalMsg = String.format( + "try visit http://%s:%d/QueryProfile/%s, show query/load profile syntax is a deprecated feature", + selfHost, httpPort, this.loadIdPath); + throw new UserException(terminalMsg); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowLoadProfileCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 4383ebc01a0b9e8..1c2a49b1b4e757e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -75,6 +75,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowFrontendsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowGrantsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowLastInsertCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowLoadProfileCommand; import org.apache.doris.nereids.trees.plans.commands.ShowPartitionIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowPluginsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowPrivilegesCommand; @@ -402,6 +403,10 @@ default R visitShowBrokerCommand(ShowBrokerCommand showBrokerCommand, C context) return visitCommand(showBrokerCommand, context); } + default R visitShowLoadProfileCommand(ShowLoadProfileCommand showLoadProfileCommand, C context) { + return visitCommand(showLoadProfileCommand, context); + } + default R visitDropRoleCommand(DropRoleCommand dropRoleCommand, C context) { return visitCommand(dropRoleCommand, context); } diff --git a/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy b/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy index e994e22ec997096..998033deff9863b 100644 --- a/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy +++ b/regression-test/suites/nereids_p0/show/test_show_commands_nereids.groovy @@ -26,5 +26,5 @@ suite("test_show_commands_nereids") { checkNereidsExecute("""show backends;""") checkNereidsExecute("""show whitelist;""") checkNereidsExecute("""show triggers;""") - + checkNereidsExecute("""show load profile \"\\";""") } From 8b9809e1c02c942f6fffa52e18bbb8b35ee21829 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Tue, 26 Nov 2024 13:31:39 +0530 Subject: [PATCH 099/110] [Enhancement] (nereids)implement showSmallFilesCommand in nereids (#44453) Issue Number: close #42770 --- .../org/apache/doris/nereids/DorisParser.g4 | 4 +- .../nereids/parser/LogicalPlanBuilder.java | 12 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/ShowSmallFilesCommand.java | 91 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../auth_call/test_ddl_file_auth.groovy | 2 + 6 files changed, 113 insertions(+), 2 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowSmallFilesCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 30b924ed878a5c3..f4394fa02c60f38 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -221,7 +221,8 @@ supportedShowStatement | SHOW ROLES #showRoles | SHOW PARTITION partitionId=INTEGER_VALUE #showPartitionId | SHOW PRIVILEGES #showPrivileges - | SHOW PROC path=STRING_LITERAL #showProc + | SHOW PROC path=STRING_LITERAL #showProc + | SHOW FILE ((FROM | IN) database=multipartIdentifier)? #showSmallFiles | SHOW STORAGE? ENGINES #showStorageEngines | SHOW CREATE CATALOG name=identifier #showCreateCatalog | SHOW SQL_BLOCK_RULE (FOR ruleName=identifier)? #showSqlBlockRule @@ -325,7 +326,6 @@ unsupportedShowStatement ((FROM | IN) database=multipartIdentifier)? wildWhere? #showFunctions | SHOW GLOBAL FULL? FUNCTIONS wildWhere? #showGlobalFunctions | SHOW TYPECAST ((FROM | IN) database=multipartIdentifier)? #showTypeCast - | SHOW FILE ((FROM | IN) database=multipartIdentifier)? #showSmallFiles | SHOW (KEY | KEYS | INDEX | INDEXES) (FROM |IN) tableName=multipartIdentifier ((FROM | IN) database=multipartIdentifier)? #showIndex diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index b77c141e2fd6959..ab4ecb44fd44253 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -224,6 +224,7 @@ import org.apache.doris.nereids.DorisParser.ShowProcedureStatusContext; import org.apache.doris.nereids.DorisParser.ShowRepositoriesContext; import org.apache.doris.nereids.DorisParser.ShowRolesContext; +import org.apache.doris.nereids.DorisParser.ShowSmallFilesContext; import org.apache.doris.nereids.DorisParser.ShowSqlBlockRuleContext; import org.apache.doris.nereids.DorisParser.ShowStorageEnginesContext; import org.apache.doris.nereids.DorisParser.ShowTableIdContext; @@ -492,6 +493,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowProcedureStatusCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRepositoriesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRolesCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowSmallFilesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.ShowStorageEnginesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; @@ -4232,6 +4234,16 @@ public LogicalPlan visitShowPlugins(ShowPluginsContext ctx) { return new ShowPluginsCommand(); } + @Override + public LogicalPlan visitShowSmallFiles(ShowSmallFilesContext ctx) { + String dbName = null; + if (ctx.database != null) { + List nameParts = visitMultipartIdentifier(ctx.database); + dbName = nameParts.get(0); // only one entry possible + } + return new ShowSmallFilesCommand(dbName); + } + @Override public LogicalPlan visitShowSqlBlockRule(ShowSqlBlockRuleContext ctx) { String ruleName = null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index e649a905b4a0c78..8698fd9855a9dea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -203,6 +203,7 @@ public enum PlanType { SHOW_PRIVILEGES_COMMAND, SHOW_REPOSITORIES_COMMAND, SHOW_ROLE_COMMAND, + SHOW_SMALL_FILES_COMMAND, SHOW_STORAGE_ENGINES_COMMAND, SHOW_TABLE_ID_COMMAND, SHOW_TRIGGERS_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowSmallFilesCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowSmallFilesCommand.java new file mode 100644 index 000000000000000..608fc8d4eff59df --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowSmallFilesCommand.java @@ -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. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Strings; + +import java.util.List; + +/** + * show small file command + */ +public class ShowSmallFilesCommand extends ShowCommand { + private static final ShowResultSetMetaData META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("Id", ScalarType.createVarchar(32))) + .addColumn(new Column("DbName", ScalarType.createVarchar(256))) + .addColumn(new Column("Catalog", ScalarType.createVarchar(32))) + .addColumn(new Column("FileName", ScalarType.createVarchar(16))) + .addColumn(new Column("FileSize", ScalarType.createVarchar(16))) + .addColumn(new Column("IsContent", ScalarType.createVarchar(16))) + .addColumn(new Column("MD5", ScalarType.createVarchar(16))) + .build(); + private String dbName; // if empty we will use current db; + + /** + * constructor + */ + public ShowSmallFilesCommand(String dbName) { + super(PlanType.SHOW_SMALL_FILES_COMMAND); + this.dbName = dbName; + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + if (Strings.isNullOrEmpty(dbName)) { + dbName = ctx.getDatabase(); + if (Strings.isNullOrEmpty(dbName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); + } + } + if (!Env.getCurrentEnv().getAccessManager() + .checkDbPriv(ConnectContext.get(), InternalCatalog.INTERNAL_CATALOG_NAME, dbName, PrivPredicate.SHOW)) { + ErrorReport.reportAnalysisException( + ErrorCode.ERR_DBACCESS_DENIED_ERROR, ConnectContext.get().getQualifiedUser(), dbName); + } + + List> results; + try { + results = Env.getCurrentEnv().getSmallFileMgr().getInfo(dbName); + } catch (DdlException e) { + throw new AnalysisException(e.getMessage()); + } + return new ShowResultSet(META_DATA, results); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowSmallFilesCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 1c2a49b1b4e757e..2e7108613c22976 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -83,6 +83,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowProcedureStatusCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRepositoriesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRolesCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowSmallFilesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.ShowStorageEnginesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; @@ -330,6 +331,10 @@ default R visitShowCreateTableCommand(ShowCreateTableCommand showCreateTableComm return visitCommand(showCreateTableCommand, context); } + default R visitShowSmallFilesCommand(ShowSmallFilesCommand showSmallFilesCommand, C context) { + return visitCommand(showSmallFilesCommand, context); + } + default R visitShowSqlBlockRuleCommand(ShowSqlBlockRuleCommand showblockruleCommand, C context) { return visitCommand(showblockruleCommand, context); } diff --git a/regression-test/suites/auth_call/test_ddl_file_auth.groovy b/regression-test/suites/auth_call/test_ddl_file_auth.groovy index 77ca5e6703f977c..99b71eabddf1405 100644 --- a/regression-test/suites/auth_call/test_ddl_file_auth.groovy +++ b/regression-test/suites/auth_call/test_ddl_file_auth.groovy @@ -76,6 +76,8 @@ suite("test_ddl_file_auth","p0,auth_call") { "catalog" = "internal" );""" sql """use ${dbName}""" + checkNereidsExecute("SHOW FILE;") + checkNereidsExecute("SHOW FILE FROM ${dbName};") def res = sql """SHOW FILE FROM ${dbName};""" assertTrue(res.size() == 1) From 78717e32bca0b9220f8c8e8dd86416ebd916daeb Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Tue, 26 Nov 2024 13:32:44 +0530 Subject: [PATCH 100/110] [Enhancement] (nereids)implement DropWorkloadGroupCommand in nereids (#44482) Issue Number: close #42621 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 7 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/DropWorkloadGroupCommand.java | 58 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../workloadgroup/WorkloadGroupMgr.java | 7 ++- .../test_nereids_workload_test.out | 6 ++ .../test_nereids_workload_test.groovy | 27 +++++++++ 8 files changed, 110 insertions(+), 3 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropWorkloadGroupCommand.java create mode 100644 regression-test/data/workload_manager_p0/test_nereids_workload_test.out create mode 100644 regression-test/suites/workload_manager_p0/test_nereids_workload_test.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index f4394fa02c60f38..c5a3b7d6efb0e8c 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -200,6 +200,7 @@ supportedDropStatement | DROP ROLE (IF EXISTS)? name=identifier #dropRole | DROP SQL_BLOCK_RULE (IF EXISTS)? identifierSeq #dropSqlBlockRule | DROP USER (IF EXISTS)? userIdentify #dropUser + | DROP WORKLOAD GROUP (IF EXISTS)? name=identifierOrText #dropWorkloadGroup ; supportedShowStatement @@ -671,7 +672,6 @@ unsupportedDropStatement ((FROM | IN) database=identifier)? properties=propertyClause #dropFile | DROP INDEX (IF EXISTS)? name=identifier ON tableName=multipartIdentifier #dropIndex | DROP RESOURCE (IF EXISTS)? name=identifierOrText #dropResource - | DROP WORKLOAD GROUP (IF EXISTS)? name=identifierOrText #dropWorkloadGroup | DROP WORKLOAD POLICY (IF EXISTS)? name=identifierOrText #dropWorkloadPolicy | DROP ENCRYPTKEY (IF EXISTS)? name=multipartIdentifier #dropEncryptkey | DROP ROW POLICY (IF EXISTS)? policyName=identifier diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index ab4ecb44fd44253..537ff71320b1e1c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -104,6 +104,7 @@ import org.apache.doris.nereids.DorisParser.DropRoleContext; import org.apache.doris.nereids.DorisParser.DropSqlBlockRuleContext; import org.apache.doris.nereids.DorisParser.DropUserContext; +import org.apache.doris.nereids.DorisParser.DropWorkloadGroupContext; import org.apache.doris.nereids.DorisParser.ElementAtContext; import org.apache.doris.nereids.DorisParser.ExceptContext; import org.apache.doris.nereids.DorisParser.ExceptOrReplaceContext; @@ -453,6 +454,7 @@ import org.apache.doris.nereids.trees.plans.commands.DropRoleCommand; import org.apache.doris.nereids.trees.plans.commands.DropSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.DropUserCommand; +import org.apache.doris.nereids.trees.plans.commands.DropWorkloadGroupCommand; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; import org.apache.doris.nereids.trees.plans.commands.ExportCommand; @@ -4374,6 +4376,11 @@ public LogicalPlan visitDropUser(DropUserContext ctx) { return new DropUserCommand(userIdent, ctx.EXISTS() != null); } + @Override + public LogicalPlan visitDropWorkloadGroup(DropWorkloadGroupContext ctx) { + return new DropWorkloadGroupCommand(ctx.name.getText(), ctx.EXISTS() != null); + } + @Override public LogicalPlan visitShowTableId(ShowTableIdContext ctx) { long tableId = -1; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 8698fd9855a9dea..8ce5caac661c921 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -184,6 +184,7 @@ public enum PlanType { EXECUTE_COMMAND, DROP_SQL_BLOCK_RULE_COMMAND, DROP_USER_COMMAND, + DROP_WORKLOAD_GROUP_NAME, SHOW_BACKENDS_COMMAND, SHOW_BLOCK_RULE_COMMAND, SHOW_BROKER_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropWorkloadGroupCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropWorkloadGroupCommand.java new file mode 100644 index 000000000000000..694fc1411db127b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropWorkloadGroupCommand.java @@ -0,0 +1,58 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +/** + * drop workload group command + */ +public class DropWorkloadGroupCommand extends DropCommand { + private final boolean ifExists; + private final String workloadGroupName; + + /** + * constructor + */ + public DropWorkloadGroupCommand(String workloadGroupName, boolean ifExists) { + super(PlanType.DROP_WORKLOAD_GROUP_NAME); + this.workloadGroupName = workloadGroupName; + this.ifExists = ifExists; + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + Env.getCurrentEnv().getWorkloadGroupMgr().dropWorkloadGroup(workloadGroupName, ifExists); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitDropWorkloadGroupCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 2e7108613c22976..9e451fac4f13714 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -44,6 +44,7 @@ import org.apache.doris.nereids.trees.plans.commands.DropRoleCommand; import org.apache.doris.nereids.trees.plans.commands.DropSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.DropUserCommand; +import org.apache.doris.nereids.trees.plans.commands.DropWorkloadGroupCommand; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand; import org.apache.doris.nereids.trees.plans.commands.ExportCommand; import org.apache.doris.nereids.trees.plans.commands.LoadCommand; @@ -424,6 +425,10 @@ default R visitDropUserCommand(DropUserCommand dropUserCommand, C context) { return visitCommand(dropUserCommand, context); } + default R visitDropWorkloadGroupCommand(DropWorkloadGroupCommand dropWorkloadGroupCommand, C context) { + return visitCommand(dropWorkloadGroupCommand, context); + } + default R visitShowTableIdCommand(ShowTableIdCommand showTableIdCommand, C context) { return visitCommand(showTableIdCommand, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java index 5e61f48bec6f2e2..b4c25524c3a8fa4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java @@ -492,7 +492,10 @@ public void alterWorkloadGroup(AlterWorkloadGroupStmt stmt) throws DdlException } public void dropWorkloadGroup(DropWorkloadGroupStmt stmt) throws DdlException { - String workloadGroupName = stmt.getWorkloadGroupName(); + dropWorkloadGroup(stmt.getWorkloadGroupName(), stmt.isIfExists()); + } + + public void dropWorkloadGroup(String workloadGroupName, boolean ifExists) throws DdlException { if (DEFAULT_GROUP_NAME.equals(workloadGroupName) || INTERNAL_GROUP_NAME.equals(workloadGroupName)) { throw new DdlException("Dropping workload group " + workloadGroupName + " is not allowed"); } @@ -521,7 +524,7 @@ public void dropWorkloadGroup(DropWorkloadGroupStmt stmt) throws DdlException { writeLock(); try { if (!nameToWorkloadGroup.containsKey(workloadGroupName)) { - if (stmt.isIfExists()) { + if (ifExists) { return; } throw new DdlException("workload group " + workloadGroupName + " does not exist"); diff --git a/regression-test/data/workload_manager_p0/test_nereids_workload_test.out b/regression-test/data/workload_manager_p0/test_nereids_workload_test.out new file mode 100644 index 000000000000000..c188c28c35c3d42 --- /dev/null +++ b/regression-test/data/workload_manager_p0/test_nereids_workload_test.out @@ -0,0 +1,6 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !check_workload_check1 -- +test_nereids_wg1 + +-- !check_workload_check2 -- + diff --git a/regression-test/suites/workload_manager_p0/test_nereids_workload_test.groovy b/regression-test/suites/workload_manager_p0/test_nereids_workload_test.groovy new file mode 100644 index 000000000000000..3f7c55f5c99cf91 --- /dev/null +++ b/regression-test/suites/workload_manager_p0/test_nereids_workload_test.groovy @@ -0,0 +1,27 @@ +// 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_nereids_workload_test") { + sql "drop workload group if exists test_nereids_wg1;" + sql "drop workload group if exists test_nereids_wg2;" + sql "create workload group test_nereids_wg1 properties('cpu_share'='1024');" + sql "create workload group test_nereids_wg2 properties('cpu_share'='1024');" + qt_check_workload_check1("select NAME from information_schema.workload_groups where NAME='test_nereids_wg1';") + checkNereidsExecute("drop workload group test_nereids_wg1;") + qt_check_workload_check2("select NAME from information_schema.workload_groups where NAME='test_nereids_wg1';") + checkNereidsExecute("drop workload group if exists test_nereids_wg2;") +} \ No newline at end of file From 1b8f7624e7a6a9c12c549157880ed81f698aa55d Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Tue, 26 Nov 2024 13:34:15 +0530 Subject: [PATCH 101/110] [Enhancement] (nereids)implement alterWorkloadGroupCommand in nereids (#44500) Issue Number: close #42793 --- .../org/apache/doris/nereids/DorisParser.g4 | 4 +- .../nereids/parser/LogicalPlanBuilder.java | 10 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/AlterWorkloadGroupCommand.java | 81 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../workloadgroup/WorkloadGroupMgr.java | 6 +- .../test_nereids_workload_alter_test.out | 7 ++ .../test_nereids_workload_alter_test.groovy | 25 ++++++ 8 files changed, 135 insertions(+), 4 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterWorkloadGroupCommand.java create mode 100644 regression-test/data/workload_manager_p0/test_nereids_workload_alter_test.out create mode 100644 regression-test/suites/workload_manager_p0/test_nereids_workload_alter_test.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index c5a3b7d6efb0e8c..c662ee295b6f49f 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -193,6 +193,8 @@ supportedAlterStatement AS query #alterView | ALTER STORAGE VAULT name=multipartIdentifier properties=propertyClause #alterStorageVault | ALTER ROLE role=identifier commentSpec #alterRole + | ALTER WORKLOAD GROUP name=identifierOrText + properties=propertyClause? #alterWorkloadGroup ; supportedDropStatement @@ -558,8 +560,6 @@ unsupportedAlterStatement | ALTER RESOURCE name=identifierOrText properties=propertyClause? #alterResource | ALTER COLOCATE GROUP name=multipartIdentifier SET LEFT_PAREN propertyItemList RIGHT_PAREN #alterColocateGroup - | ALTER WORKLOAD GROUP name=identifierOrText - properties=propertyClause? #alterWorkloadGroup | ALTER WORKLOAD POLICY name=identifierOrText properties=propertyClause? #alterWorkloadPolicy | ALTER ROUTINE LOAD FOR name=multipartIdentifier properties=propertyClause? diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 537ff71320b1e1c..dc0fe2aa4a10d6b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -56,6 +56,7 @@ import org.apache.doris.nereids.DorisParser.AlterRoleContext; import org.apache.doris.nereids.DorisParser.AlterStorageVaultContext; import org.apache.doris.nereids.DorisParser.AlterViewContext; +import org.apache.doris.nereids.DorisParser.AlterWorkloadGroupContext; import org.apache.doris.nereids.DorisParser.ArithmeticBinaryContext; import org.apache.doris.nereids.DorisParser.ArithmeticUnaryContext; import org.apache.doris.nereids.DorisParser.ArrayLiteralContext; @@ -430,6 +431,7 @@ import org.apache.doris.nereids.trees.plans.commands.AlterRoleCommand; import org.apache.doris.nereids.trees.plans.commands.AlterStorageVaultCommand; import org.apache.doris.nereids.trees.plans.commands.AlterViewCommand; +import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadGroupCommand; import org.apache.doris.nereids.trees.plans.commands.CallCommand; import org.apache.doris.nereids.trees.plans.commands.CancelJobTaskCommand; import org.apache.doris.nereids.trees.plans.commands.CancelMTMVTaskCommand; @@ -4298,12 +4300,20 @@ public LogicalPlan visitShowCreateMaterializedView(ShowCreateMaterializedViewCon return new ShowCreateMaterializedViewCommand(stripQuotes(ctx.mvName.getText()), new TableNameInfo(nameParts)); } + @Override + public LogicalPlan visitAlterWorkloadGroup(AlterWorkloadGroupContext ctx) { + Map properties = ctx.propertyClause() != null + ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) : Maps.newHashMap(); + return new AlterWorkloadGroupCommand(ctx.name.getText(), properties); + } + @Override public LogicalPlan visitAlterRole(AlterRoleContext ctx) { String comment = visitCommentSpec(ctx.commentSpec()); return new AlterRoleCommand(ctx.role.getText(), comment); } + @Override public LogicalPlan visitShowFrontends(ShowFrontendsContext ctx) { String detail = (ctx.name != null) ? ctx.name.getText() : null; return new ShowFrontendsCommand(detail); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 8ce5caac661c921..04551da904bfa9c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -170,6 +170,7 @@ public enum PlanType { ALTER_ROLE_COMMAND, ALTER_VIEW_COMMAND, ALTER_STORAGE_VAULT, + ALTER_WORKLOAD_GROUP_COMMAND, DROP_CATALOG_RECYCLE_BIN_COMMAND, UNSET_VARIABLE_COMMAND, UNSET_DEFAULT_STORAGE_VAULT_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterWorkloadGroupCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterWorkloadGroupCommand.java new file mode 100644 index 000000000000000..8cceaa20fdce9aa --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterWorkloadGroupCommand.java @@ -0,0 +1,81 @@ +// 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.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.resource.workloadgroup.WorkloadGroup; +import org.apache.doris.resource.workloadgroup.WorkloadGroupMgr; + +import org.apache.commons.lang3.StringUtils; + +import java.util.Map; + +/** + * alter workload group command + */ +public class AlterWorkloadGroupCommand extends AlterCommand { + private final String workloadGroupName; + private final Map properties; + + /** + * constructor + */ + public AlterWorkloadGroupCommand(String workloadGroupName, Map properties) { + super(PlanType.ALTER_WORKLOAD_GROUP_COMMAND); + this.workloadGroupName = workloadGroupName; + this.properties = properties; + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + + if (properties == null || properties.isEmpty()) { + throw new AnalysisException("Workload Group properties can't be empty"); + } + + if (properties.containsKey(WorkloadGroup.INTERNAL_TYPE)) { + throw new AnalysisException(WorkloadGroup.INTERNAL_TYPE + " can not be create or modified "); + } + String tagStr = properties.get(WorkloadGroup.TAG); + if (!StringUtils.isEmpty(tagStr) && (WorkloadGroupMgr.DEFAULT_GROUP_NAME.equals(workloadGroupName) + || WorkloadGroupMgr.INTERNAL_GROUP_NAME.equals(workloadGroupName))) { + throw new AnalysisException( + WorkloadGroupMgr.INTERNAL_GROUP_NAME + " and " + WorkloadGroupMgr.DEFAULT_GROUP_NAME + + " group can not set tag"); + } + + Env.getCurrentEnv().getWorkloadGroupMgr().alterWorkloadGroup(workloadGroupName, properties); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitAlterWorkloadGroupCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 9e451fac4f13714..13eeef182641fb9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -22,6 +22,7 @@ import org.apache.doris.nereids.trees.plans.commands.AlterMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.AlterRoleCommand; import org.apache.doris.nereids.trees.plans.commands.AlterViewCommand; +import org.apache.doris.nereids.trees.plans.commands.AlterWorkloadGroupCommand; import org.apache.doris.nereids.trees.plans.commands.CallCommand; import org.apache.doris.nereids.trees.plans.commands.CancelJobTaskCommand; import org.apache.doris.nereids.trees.plans.commands.CancelMTMVTaskCommand; @@ -377,6 +378,10 @@ default R visitAlterRoleCommand(AlterRoleCommand alterRoleCommand, C context) { return visitCommand(alterRoleCommand, context); } + default R visitAlterWorkloadGroupCommand(AlterWorkloadGroupCommand alterWorkloadGroupCommand, C context) { + return visitCommand(alterWorkloadGroupCommand, context); + } + default R visitCleanAllProfileCommand(CleanAllProfileCommand cleanAllProfileCommand, C context) { return visitCommand(cleanAllProfileCommand, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java index b4c25524c3a8fa4..22cbcf4a0a15023 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java @@ -464,8 +464,10 @@ private void checkGlobalUnlock(WorkloadGroup newWg, WorkloadGroup oldWg) throws } public void alterWorkloadGroup(AlterWorkloadGroupStmt stmt) throws DdlException { - String workloadGroupName = stmt.getWorkloadGroupName(); - Map properties = stmt.getProperties(); + alterWorkloadGroup(stmt.getWorkloadGroupName(), stmt.getProperties()); + } + + public void alterWorkloadGroup(String workloadGroupName, Map properties) throws DdlException { if (properties.size() == 0) { throw new DdlException("alter workload group should contain at least one property"); } diff --git a/regression-test/data/workload_manager_p0/test_nereids_workload_alter_test.out b/regression-test/data/workload_manager_p0/test_nereids_workload_alter_test.out new file mode 100644 index 000000000000000..897cef4c230a514 --- /dev/null +++ b/regression-test/data/workload_manager_p0/test_nereids_workload_alter_test.out @@ -0,0 +1,7 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !check_workload_check1 -- +1024 10 + +-- !check_workload_check2 -- +20 8 + diff --git a/regression-test/suites/workload_manager_p0/test_nereids_workload_alter_test.groovy b/regression-test/suites/workload_manager_p0/test_nereids_workload_alter_test.groovy new file mode 100644 index 000000000000000..b3f600b3f186b44 --- /dev/null +++ b/regression-test/suites/workload_manager_p0/test_nereids_workload_alter_test.groovy @@ -0,0 +1,25 @@ +// 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_nereids_workload_test") { + sql "drop workload group if exists test_nereids_alter_wg1;" + sql "create workload group test_nereids_alter_wg1 properties('cpu_share'='1024', 'scan_thread_num'='10');" + qt_check_workload_check1("select CPU_SHARE,scan_thread_num from information_schema.workload_groups where NAME='test_nereids_alter_wg1';") + checkNereidsExecute("Alter workload group test_nereids_alter_wg1 properties('cpu_share'='20', 'scan_thread_num'='8');") + qt_check_workload_check2("select CPU_SHARE,scan_thread_num from information_schema.workload_groups where NAME='test_nereids_alter_wg1';") + checkNereidsExecute("drop workload group if exists test_nereids_alter_wg1;") +} \ No newline at end of file From 95fdd45600b4357a0ea8b6a0d1c0ac6182dd0500 Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Tue, 26 Nov 2024 13:35:46 +0530 Subject: [PATCH 102/110] [Enhancement] (nereids)implement DropEncryptKeyCommand in nereids (#44506) Issue Number: close #42623 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../apache/doris/analysis/EncryptKeyName.java | 25 +++++++ .../nereids/parser/LogicalPlanBuilder.java | 9 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/DropEncryptkeyCommand.java | 65 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../nereids_p0/test_nereids_encrypt_test.out | 8 +++ .../test_nereids_encrypt_test.groovy | 29 +++++++++ 8 files changed, 143 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropEncryptkeyCommand.java create mode 100644 regression-test/data/nereids_p0/test_nereids_encrypt_test.out create mode 100644 regression-test/suites/nereids_p0/test_nereids_encrypt_test.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index c662ee295b6f49f..932f644fca4e7aa 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -199,6 +199,7 @@ supportedAlterStatement supportedDropStatement : DROP CATALOG RECYCLE BIN WHERE idType=STRING_LITERAL EQ id=INTEGER_VALUE #dropCatalogRecycleBin + | DROP ENCRYPTKEY (IF EXISTS)? name=multipartIdentifier #dropEncryptkey | DROP ROLE (IF EXISTS)? name=identifier #dropRole | DROP SQL_BLOCK_RULE (IF EXISTS)? identifierSeq #dropSqlBlockRule | DROP USER (IF EXISTS)? userIdentify #dropUser @@ -673,7 +674,6 @@ unsupportedDropStatement | DROP INDEX (IF EXISTS)? name=identifier ON tableName=multipartIdentifier #dropIndex | DROP RESOURCE (IF EXISTS)? name=identifierOrText #dropResource | DROP WORKLOAD POLICY (IF EXISTS)? name=identifierOrText #dropWorkloadPolicy - | DROP ENCRYPTKEY (IF EXISTS)? name=multipartIdentifier #dropEncryptkey | DROP ROW POLICY (IF EXISTS)? policyName=identifier ON tableName=multipartIdentifier (FOR (userIdentify | ROLE roleName=identifier))? #dropRowPolicy diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/EncryptKeyName.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/EncryptKeyName.java index 155f1753bf4a349..e76bfb7e5b20a2c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/EncryptKeyName.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/EncryptKeyName.java @@ -25,6 +25,7 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.persist.gson.GsonUtils; +import org.apache.doris.qe.ConnectContext; import com.google.common.base.Strings; import com.google.gson.annotations.SerializedName; @@ -34,6 +35,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.List; import java.util.Objects; public class EncryptKeyName implements Writable { @@ -52,6 +54,19 @@ public EncryptKeyName(String db, String keyName) { } } + /** + * EncryptKeyName + * @param parts like [db1,keyName] or [keyName] + */ + public EncryptKeyName(List parts) { + int size = parts.size(); + keyName = parts.get(size - 1); + keyName = keyName.toLowerCase(); + if (size >= 2) { + db = parts.get(size - 2); + } + } + public EncryptKeyName(String keyName) { this.db = null; this.keyName = keyName.toLowerCase(); @@ -67,6 +82,16 @@ public void analyze(Analyzer analyzer) throws AnalysisException { } } + public void analyze(ConnectContext ctx) throws AnalysisException { + FeNameFormat.checkCommonName("EncryptKey", keyName); + if (db == null) { + db = ctx.getDatabase(); + if (Strings.isNullOrEmpty(db)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); + } + } + } + public String getDb() { return db; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index dc0fe2aa4a10d6b..320043e12b8b534 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -20,6 +20,7 @@ import org.apache.doris.analysis.ArithmeticExpr.Operator; import org.apache.doris.analysis.BrokerDesc; import org.apache.doris.analysis.ColumnNullableType; +import org.apache.doris.analysis.EncryptKeyName; import org.apache.doris.analysis.PassVar; import org.apache.doris.analysis.SetType; import org.apache.doris.analysis.StorageBackend; @@ -100,6 +101,7 @@ import org.apache.doris.nereids.DorisParser.DereferenceContext; import org.apache.doris.nereids.DorisParser.DropCatalogRecycleBinContext; import org.apache.doris.nereids.DorisParser.DropConstraintContext; +import org.apache.doris.nereids.DorisParser.DropEncryptkeyContext; import org.apache.doris.nereids.DorisParser.DropMTMVContext; import org.apache.doris.nereids.DorisParser.DropProcedureContext; import org.apache.doris.nereids.DorisParser.DropRoleContext; @@ -450,6 +452,7 @@ import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand; import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand.IdType; import org.apache.doris.nereids.trees.plans.commands.DropConstraintCommand; +import org.apache.doris.nereids.trees.plans.commands.DropEncryptkeyCommand; import org.apache.doris.nereids.trees.plans.commands.DropJobCommand; import org.apache.doris.nereids.trees.plans.commands.DropMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.DropProcedureCommand; @@ -4375,6 +4378,12 @@ public LogicalPlan visitDropRole(DropRoleContext ctx) { return new DropRoleCommand(ctx.name.getText(), ctx.EXISTS() != null); } + @Override + public LogicalPlan visitDropEncryptkey(DropEncryptkeyContext ctx) { + List nameParts = visitMultipartIdentifier(ctx.name); + return new DropEncryptkeyCommand(new EncryptKeyName(nameParts), ctx.EXISTS() != null); + } + @Override public LogicalPlan visitDropSqlBlockRule(DropSqlBlockRuleContext ctx) { return new DropSqlBlockRuleCommand(visitIdentifierSeq(ctx.identifierSeq()), ctx.EXISTS() != null); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 04551da904bfa9c..1f7b838edfd9059 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -172,6 +172,7 @@ public enum PlanType { ALTER_STORAGE_VAULT, ALTER_WORKLOAD_GROUP_COMMAND, DROP_CATALOG_RECYCLE_BIN_COMMAND, + DROP_ENCRYPTKEY_COMMAND, UNSET_VARIABLE_COMMAND, UNSET_DEFAULT_STORAGE_VAULT_COMMAND, UNSUPPORTED_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropEncryptkeyCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropEncryptkeyCommand.java new file mode 100644 index 000000000000000..4c163d342d696ba --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropEncryptkeyCommand.java @@ -0,0 +1,65 @@ +// 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.nereids.trees.plans.commands; + +import org.apache.doris.analysis.EncryptKeyName; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.EncryptKeySearchDesc; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +/** + * drop encrypt key command + */ +public class DropEncryptkeyCommand extends DropCommand { + private final boolean ifExists; + private final EncryptKeyName encryptKeyName; + + /** + * constructor + */ + public DropEncryptkeyCommand(EncryptKeyName encryptKeyName, boolean ifExists) { + super(PlanType.DROP_ENCRYPTKEY_COMMAND); + this.encryptKeyName = encryptKeyName; + this.ifExists = ifExists; + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check operation privilege + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + // analyze encryptkey name + encryptKeyName.analyze(ctx); + EncryptKeySearchDesc encryptKeySearchDesc = new EncryptKeySearchDesc(encryptKeyName); + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(encryptKeyName.getDb()); + db.dropEncryptKey(encryptKeySearchDesc, ifExists); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitDropEncryptKeyCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 13eeef182641fb9..6b801524fb295f3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -39,6 +39,7 @@ import org.apache.doris.nereids.trees.plans.commands.DeleteFromUsingCommand; import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand; import org.apache.doris.nereids.trees.plans.commands.DropConstraintCommand; +import org.apache.doris.nereids.trees.plans.commands.DropEncryptkeyCommand; import org.apache.doris.nereids.trees.plans.commands.DropJobCommand; import org.apache.doris.nereids.trees.plans.commands.DropMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.DropProcedureCommand; @@ -422,6 +423,10 @@ default R visitDropRoleCommand(DropRoleCommand dropRoleCommand, C context) { return visitCommand(dropRoleCommand, context); } + default R visitDropEncryptKeyCommand(DropEncryptkeyCommand dropEncryptkeyCommand, C context) { + return visitCommand(dropEncryptkeyCommand, context); + } + default R visitDropSqlBlockRuleCommand(DropSqlBlockRuleCommand dropSqlBlockRuleCommand, C context) { return visitCommand(dropSqlBlockRuleCommand, context); } diff --git a/regression-test/data/nereids_p0/test_nereids_encrypt_test.out b/regression-test/data/nereids_p0/test_nereids_encrypt_test.out new file mode 100644 index 000000000000000..7984f572e8f26ff --- /dev/null +++ b/regression-test/data/nereids_p0/test_nereids_encrypt_test.out @@ -0,0 +1,8 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !check_encrypt_1 -- +test_nereids_encrypt_test_db.test_nereids_encrypt_test_key ABCD123456789 + +-- !check_encrypt_2 -- + +-- !check_encrypt_3 -- + diff --git a/regression-test/suites/nereids_p0/test_nereids_encrypt_test.groovy b/regression-test/suites/nereids_p0/test_nereids_encrypt_test.groovy new file mode 100644 index 000000000000000..2fab616580faaad --- /dev/null +++ b/regression-test/suites/nereids_p0/test_nereids_encrypt_test.groovy @@ -0,0 +1,29 @@ +// 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_nereids_encrypt_test") { + def dbName="test_nereids_encrypt_test_db" + def encryptkeyName="test_nereids_encrypt_test_key" + sql """ create database IF NOT EXISTS ${dbName}; """ + sql """ use ${dbName}; """ + checkNereidsExecute("drop encryptkey if exists ${encryptkeyName}") + sql """CREATE ENCRYPTKEY ${encryptkeyName} AS "ABCD123456789";""" + qt_check_encrypt_1("SHOW ENCRYPTKEYS FROM ${dbName}") + checkNereidsExecute("drop encryptkey ${encryptkeyName}") + qt_check_encrypt_2("SHOW ENCRYPTKEYS FROM ${dbName}") + checkNereidsExecute("drop encryptkey if exists ${encryptkeyName}") + qt_check_encrypt_3("SHOW ENCRYPTKEYS FROM ${dbName}") +} \ No newline at end of file From 8751287549245e5bd6ac8adbe21d75af6c9063ed Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Tue, 26 Nov 2024 17:03:04 +0800 Subject: [PATCH 103/110] [chore](github)Update .asf.yaml (#44614) --- .asf.yaml | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index 3892aca2eddb77c..f4c21ce4b0f3c99 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -56,8 +56,7 @@ github: - cloud_p0 (Doris Cloud Regression) - FE UT (Doris FE UT) - BE UT (Doris BE UT) - - Build Broker - - ShellCheck + - Build Broker - Build Third Party Libraries (Linux) - Build Third Party Libraries (macOS) - Build Third Party Libraries (macOS-arm64) @@ -80,7 +79,6 @@ github: - Clang Formatter - CheckStyle - Build Broker - - ShellCheck - Build Third Party Libraries (Linux) - Build Third Party Libraries (macOS) - FE UT (Doris FE UT) @@ -103,7 +101,6 @@ github: - Clang Formatter - CheckStyle - Build Broker - - ShellCheck - Build Third Party Libraries (Linux) - Build Third Party Libraries (macOS) - COMPILE (DORIS_COMPILE) From aa15ca692a1cc010235e15571fac29699d26a367 Mon Sep 17 00:00:00 2001 From: Mryange Date: Tue, 26 Nov 2024 17:32:12 +0800 Subject: [PATCH 104/110] [fix](DECIMAL) error DECIMAL cat to BOOLEAN (#44326) In the past, there were issues with converting `double` and `decimal` to `boolean`. For example, a `double` value like 0.13 would first be cast to `uint8`, resulting in 0. Then, it would be converted to `bool`, yielding 0 (incorrect, as the expected result is 1). Similarly, `decimal` values were directly cast to `uint8`, leading to non-0/1 values for `bool`. This issue arises because Doris internally uses `uint8` to represent `boolean`. before ``` mysql> select cast(40.123 as BOOLEAN); +-------------------------+ | cast(40.123 as BOOLEAN) | +-------------------------+ | 40 | +-------------------------+ ``` now ``` mysql> select cast(40.123 as BOOLEAN); +-------------------------+ | cast(40.123 as BOOLEAN) | +-------------------------+ | 1 | +-------------------------+ ``` --- be/src/vec/data_types/data_type_decimal.h | 19 ++++--- be/src/vec/functions/function_cast.h | 43 ++++++++++----- .../test_cast_decimalv3_as_bool.out | 17 ++++++ .../test_cast_decimalv3_as_bool.groovy | 55 +++++++++++++++++++ .../test_case_function_null.groovy | 18 ++++-- 5 files changed, 124 insertions(+), 28 deletions(-) create mode 100644 regression-test/data/correctness/test_cast_decimalv3_as_bool.out create mode 100644 regression-test/suites/correctness/test_cast_decimalv3_as_bool.groovy diff --git a/be/src/vec/data_types/data_type_decimal.h b/be/src/vec/data_types/data_type_decimal.h index a44390cd58d86a7..74655ff6ee8becf 100644 --- a/be/src/vec/data_types/data_type_decimal.h +++ b/be/src/vec/data_types/data_type_decimal.h @@ -460,15 +460,20 @@ void convert_from_decimals(RealTo* dst, const RealFrom* src, UInt32 precicion_fr MaxFieldType multiplier = DataTypeDecimal::get_scale_multiplier(scale_from); FromDataType from_data_type(precicion_from, scale_from); for (size_t i = 0; i < size; i++) { - auto tmp = static_cast(src[i]).value / multiplier.value; - if constexpr (narrow_integral) { - if (tmp < min_result.value || tmp > max_result.value) { - THROW_DECIMAL_CONVERT_OVERFLOW_EXCEPTION(from_data_type.to_string(src[i]), - from_data_type.get_name(), - OrigToDataType {}.get_name()); + // uint8_t now use as boolean in doris + if constexpr (std::is_same_v) { + dst[i] = static_cast(src[i]).value != 0; + } else { + auto tmp = static_cast(src[i]).value / multiplier.value; + if constexpr (narrow_integral) { + if (tmp < min_result.value || tmp > max_result.value) { + THROW_DECIMAL_CONVERT_OVERFLOW_EXCEPTION(from_data_type.to_string(src[i]), + from_data_type.get_name(), + OrigToDataType {}.get_name()); + } } + dst[i] = tmp; } - dst[i] = tmp; } } diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h index 5de820dfa3a24fe..7027a4e9c622bc1 100644 --- a/be/src/vec/functions/function_cast.h +++ b/be/src/vec/functions/function_cast.h @@ -258,6 +258,21 @@ struct ConvertImpl { using FromFieldType = typename FromDataType::FieldType; using ToFieldType = typename ToDataType::FieldType; + // `static_cast_set` is introduced to wrap `static_cast` and handle special cases. + // Doris uses `uint8` to represent boolean values internally. + // Directly `static_cast` to `uint8` can result in non-0/1 values, + // To address this, `static_cast_set` performs an additional check: + // For `uint8` types, it explicitly uses `static_cast` to ensure + // the result is either 0 or 1. + static void static_cast_set(ToFieldType& to, const FromFieldType& from) { + // uint8_t now use as boolean in doris + if constexpr (std::is_same_v) { + to = static_cast(from); + } else { + to = static_cast(from); + } + } + template static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments, uint32_t result, size_t input_rows_count, @@ -377,8 +392,9 @@ struct ConvertImpl { } else if constexpr (IsDateTimeV2Type) { DataTypeDateTimeV2::cast_from_date(vec_from[i], vec_to[i]); } else { - vec_to[i] = - reinterpret_cast(vec_from[i]).to_int64(); + static_cast_set( + vec_to[i], + reinterpret_cast(vec_from[i]).to_int64()); } } } else if constexpr (IsTimeV2Type) { @@ -409,13 +425,16 @@ struct ConvertImpl { } } else { if constexpr (IsDateTimeV2Type) { - vec_to[i] = reinterpret_cast&>( - vec_from[i]) - .to_int64(); + static_cast_set( + vec_to[i], + reinterpret_cast&>( + vec_from[i]) + .to_int64()); } else { - vec_to[i] = reinterpret_cast&>( - vec_from[i]) - .to_int64(); + static_cast_set(vec_to[i], + reinterpret_cast&>( + vec_from[i]) + .to_int64()); } } } @@ -441,16 +460,10 @@ struct ConvertImpl { } } else { for (size_t i = 0; i < size; ++i) { - vec_to[i] = static_cast(vec_from[i]); + static_cast_set(vec_to[i], vec_from[i]); } } } - // TODO: support boolean cast more reasonable - if constexpr (std::is_same_v) { - for (int i = 0; i < size; ++i) { - vec_to[i] = static_cast(vec_to[i]); - } - } block.replace_by_position(result, std::move(col_to)); } else { diff --git a/regression-test/data/correctness/test_cast_decimalv3_as_bool.out b/regression-test/data/correctness/test_cast_decimalv3_as_bool.out new file mode 100644 index 000000000000000..4f41130b00bafd7 --- /dev/null +++ b/regression-test/data/correctness/test_cast_decimalv3_as_bool.out @@ -0,0 +1,17 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select1 -- +0.000 13131.213132100 0E-16 +0.000 2131231.231000000 2.3323000E-9 +3.141 0E-9 123123.2131231231322130 + +-- !select2 -- +false true false +false true true +true false true + +-- !select3 -- +true 1 true false + +-- !select3 -- +true 1 true false + diff --git a/regression-test/suites/correctness/test_cast_decimalv3_as_bool.groovy b/regression-test/suites/correctness/test_cast_decimalv3_as_bool.groovy new file mode 100644 index 000000000000000..768da493251e6cd --- /dev/null +++ b/regression-test/suites/correctness/test_cast_decimalv3_as_bool.groovy @@ -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. + +suite("test_cast_decimalv3_as_bool") { + sql """ DROP TABLE IF EXISTS cast_decimalv3_as_bool """ + sql """ + CREATE TABLE IF NOT EXISTS cast_decimalv3_as_bool ( + `id` int(11) , + `k1` decimalv3(9,3) , + `k2` decimalv3(18,9) , + `k3` decimalv3(38,16) , + ) + UNIQUE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 10 + PROPERTIES ( + "enable_unique_key_merge_on_write" = "true", + "replication_num" = "1" + ); + """ + sql """ + set enable_nereids_planner=true,enable_fold_constant_by_be = false + """ + sql """ + INSERT INTO cast_decimalv3_as_bool VALUES + (1,0.00001,13131.2131321,0.000000000000000000), + (2,0.00000,2131231.231,0.0000000023323), + (3,3.141414,0.0000000000,123123.213123123132213); + """ + qt_select1 """ + select k1,k2,k3 from cast_decimalv3_as_bool order by id + """ + qt_select2 """ + select cast(k1 as boolean), cast(k2 as boolean) , cast(k3 as boolean) from cast_decimalv3_as_bool order by id + """ + qt_select3""" + select cast(3.00001 as boolean), cast(cast(3.00001 as boolean) as int),cast(0.001 as boolean),cast(0.000 as boolean); + """ + qt_select3""" + select cast(cast(3.00001 as double)as boolean), cast(cast(cast(3.00001 as double) as boolean) as int),cast(cast(0.001 as double) as boolean),cast(cast(0.000 as double) as boolean); + """ +} \ No newline at end of file diff --git a/regression-test/suites/query_p0/sql_functions/case_function/test_case_function_null.groovy b/regression-test/suites/query_p0/sql_functions/case_function/test_case_function_null.groovy index 5138db6e73b4add..a91c86b5f489089 100644 --- a/regression-test/suites/query_p0/sql_functions/case_function/test_case_function_null.groovy +++ b/regression-test/suites/query_p0/sql_functions/case_function/test_case_function_null.groovy @@ -185,10 +185,11 @@ suite("test_case_function_null", "query,p0,arrow_flight_sql") { c2, c1; """ - + // There is a behavior change. The 0.4cast boolean used to be 0 in the past, but now it has changed to 1. + // Therefore, we need to update the case accordingly. qt_sql_case1 """ SELECT SUM( - CASE (((NULL BETWEEN NULL AND NULL)) and (CAST(0.4716 AS BOOLEAN))) + CASE (((NULL BETWEEN NULL AND NULL)) and (CAST(0.0 AS BOOLEAN))) WHEN ((CAST('-1530390546' AS VARCHAR)) LIKE ('-1678299490')) THEN (- (+ case_null2.c0)) WHEN CASE (NULL IN (NULL)) @@ -197,9 +198,10 @@ suite("test_case_function_null", "query,p0,arrow_flight_sql") { END) FROM case_null2; """ - + // There is a behavior change. The 0.4cast boolean used to be 0 in the past, but now it has changed to 1. + // Therefore, we need to update the case accordingly. qt_sql_case2 """ - SELECT SUM(CASE (((NULL BETWEEN NULL AND NULL)) and (CAST(0.4716 AS BOOLEAN))) + SELECT SUM(CASE (((NULL BETWEEN NULL AND NULL)) and (CAST(0.0 AS BOOLEAN))) WHEN ((CAST('-1530390546' AS VARCHAR)) LIKE ('-1678299490')) THEN (- (+ case_null2.c0)) END) @@ -209,9 +211,11 @@ suite("test_case_function_null", "query,p0,arrow_flight_sql") { sql "SET experimental_enable_nereids_planner=true" sql "SET enable_fallback_to_original_planner=false" + // There is a behavior change. The 0.4cast boolean used to be 0 in the past, but now it has changed to 1. + // Therefore, we need to update the case accordingly. qt_sql_case1 """ SELECT SUM( - CASE (((NULL BETWEEN NULL AND NULL)) and (CAST(0.4716 AS BOOLEAN))) + CASE (((NULL BETWEEN NULL AND NULL)) and (CAST(0.0 AS BOOLEAN))) WHEN ((CAST('-1530390546' AS VARCHAR)) LIKE ('-1678299490')) THEN (- (+ case_null2.c0)) WHEN CASE (NULL IN (NULL)) @@ -221,8 +225,10 @@ suite("test_case_function_null", "query,p0,arrow_flight_sql") { FROM case_null2; """ + // There is a behavior change. The 0.4cast boolean used to be 0 in the past, but now it has changed to 1. + // Therefore, we need to update the case accordingly. qt_sql_case2 """ - SELECT SUM(CASE (((NULL BETWEEN NULL AND NULL)) and (CAST(0.4716 AS BOOLEAN))) + SELECT SUM(CASE (((NULL BETWEEN NULL AND NULL)) and (CAST(0.0 AS BOOLEAN))) WHEN ((CAST('-1530390546' AS VARCHAR)) LIKE ('-1678299490')) THEN (- (+ case_null2.c0)) END) From 4d48597487b35d9bece381aefae1736a9016c9f1 Mon Sep 17 00:00:00 2001 From: Sun Chenyang Date: Tue, 26 Nov 2024 17:35:42 +0800 Subject: [PATCH 105/110] [fix](inverted index) fix close error on inverted index writer (#44601) Related PR: #44205 fix close error on inverted index writer --- be/src/olap/rowset/segment_v2/inverted_index_writer.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp b/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp index 08cebaa07d6670b..86a8f89e4c94e4f 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp @@ -667,8 +667,13 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { FINALLY_CLOSE(meta_out); FINALLY_CLOSE(data_out); FINALLY_CLOSE(index_out); - FINALLY_CLOSE(_dir); - FINALLY_CLOSE(_index_writer); + if constexpr (field_is_numeric_type(field_type)) { + FINALLY_CLOSE(_dir); + } else if constexpr (field_is_slice_type(field_type)) { + FINALLY_CLOSE(_index_writer); + // After closing the _index_writer, it needs to be reset to null to prevent issues of not closing it or closing it multiple times. + _index_writer.reset(); + } }) return Status::OK(); From 14a8bac3cad8000fe96e8676e7ef04675c412716 Mon Sep 17 00:00:00 2001 From: qiye Date: Tue, 26 Nov 2024 19:20:47 +0800 Subject: [PATCH 106/110] [fix](build index)Fix non-master nodes cannot see the latest build index job status (#44544) Problem Summary: Non-master nodes cannot see the latest build index job status, this will cause the case `test_build_index_with_clone_fault_injection` unstable. We add forward_to_master in show build index statement to get the latest job status. ### Release note Fix non-master nodes cannot see the latest build index job status --- .../java/org/apache/doris/alter/IndexChangeJob.java | 1 + .../java/org/apache/doris/alter/SchemaChangeJobV2.java | 2 +- .../org/apache/doris/analysis/ShowBuildIndexStmt.java | 10 ++++++++++ .../test_build_index_with_clone_fault.groovy | 1 + 4 files changed, 13 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java index a51cdb01699e239..e00587bee16d98c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/IndexChangeJob.java @@ -348,6 +348,7 @@ protected void runWaitingTxnJob() throws AlterCancelException { olapTable.readUnlock(); } this.jobState = JobState.RUNNING; + // DO NOT write edit log here, tasks will be sent again if FE restart or master changed. LOG.info("transfer inverted index job {} state to {}", jobId, this.jobState); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java index 522433dfe898da1..58e3e05e214a9d2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java @@ -556,7 +556,7 @@ protected void runWaitingTxnJob() throws AlterCancelException { this.jobState = JobState.RUNNING; - // DO NOT write edit log here, tasks will be send again if FE restart or master changed. + // DO NOT write edit log here, tasks will be sent again if FE restart or master changed. LOG.info("transfer schema change job {} state to {}", jobId, this.jobState); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBuildIndexStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBuildIndexStmt.java index 944365fb0d48dfe..9914bb76400a41c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBuildIndexStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBuildIndexStmt.java @@ -29,6 +29,7 @@ import org.apache.doris.common.proc.ProcNodeInterface; import org.apache.doris.common.proc.ProcService; import org.apache.doris.common.util.OrderByPair; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; import com.google.common.base.Strings; @@ -224,4 +225,13 @@ public ShowResultSetMetaData getMetaData() { return builder.build(); } + + @Override + public RedirectStatus getRedirectStatus() { + if (ConnectContext.get().getSessionVariable().getForwardToMaster()) { + return RedirectStatus.FORWARD_NO_SYNC; + } else { + return RedirectStatus.NO_FORWARD; + } + } } diff --git a/regression-test/suites/fault_injection_p0/test_build_index_with_clone_fault.groovy b/regression-test/suites/fault_injection_p0/test_build_index_with_clone_fault.groovy index ec175792d48de37..998c8fa98619e0b 100644 --- a/regression-test/suites/fault_injection_p0/test_build_index_with_clone_fault.groovy +++ b/regression-test/suites/fault_injection_p0/test_build_index_with_clone_fault.groovy @@ -58,6 +58,7 @@ suite("test_build_index_with_clone_fault_injection", "nonConcurrent"){ while (attempt < maxRetries) { def show_build_index = sql_return_maparray("show build index where TableName = \"${tbl}\" ORDER BY JobId DESC LIMIT 1") if (show_build_index && show_build_index.size() > 0) { + logger.info("show build index result: ${show_build_index}") def currentState = show_build_index[0].State def currentMsg = show_build_index[0].Msg if ((currentState == expectedState && currentMsg == expectedMsg) || currentState == "FINISHED") { From 0066bcdb26f08e2e5945a4a67a91814aae657814 Mon Sep 17 00:00:00 2001 From: zfr95 Date: Tue, 26 Nov 2024 20:36:16 +0800 Subject: [PATCH 107/110] [fix](auth)fix stream load information to observer (#44632) --- .../suites/auth_call/test_dml_stream_load_auth.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression-test/suites/auth_call/test_dml_stream_load_auth.groovy b/regression-test/suites/auth_call/test_dml_stream_load_auth.groovy index 240fdde6f6926d7..30f01c0e72d9c5b 100644 --- a/regression-test/suites/auth_call/test_dml_stream_load_auth.groovy +++ b/regression-test/suites/auth_call/test_dml_stream_load_auth.groovy @@ -59,7 +59,7 @@ suite("test_dml_stream_load_auth","p0,auth_call") { def path_file = "${context.file.parent}/../../data/auth_call/stream_load_data.csv" def load_path = "${context.file.parent}/../../data/auth_call/stream_load_cm.sh" - def cm = """curl --location-trusted -u ${user}:${pwd} -H "column_separator:," -T ${path_file} http://${sql_ip}:${http_port}/api/${dbName}/${tableName}/_stream_load""" + def cm = """curl -v --location-trusted -u ${user}:${pwd} -H "column_separator:," -T ${path_file} http://${sql_ip}:${http_port}/api/${dbName}/${tableName}/_stream_load""" logger.info("cm: " + cm) write_to_file(load_path, cm) cm = "bash " + load_path From bce9b69c7da60d62101516612d34707763b77bc0 Mon Sep 17 00:00:00 2001 From: Gabriel Date: Tue, 26 Nov 2024 20:42:39 +0800 Subject: [PATCH 108/110] [fix](coordinator) Fix wrong bucket assignment (#44603) follow-up : #44459 --- .../src/main/java/org/apache/doris/qe/Coordinator.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 61ecaf7fc8b359a..3a6f6e4f84069d5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -2766,17 +2766,18 @@ private void assignScanRanges(PlanFragmentId fragmentId, int parallelExecInstanc .addAll(nodeScanRange.getValue()); } } - params.instanceExecParams.add(instanceParam); + List instanceExecParams = new ArrayList<>(); + instanceExecParams.add(instanceParam); for (int i = 1; i < parallelExecInstanceNum; i++) { - params.instanceExecParams.add(new FInstanceExecParam( + instanceExecParams.add(new FInstanceExecParam( null, addressScanRange.getKey(), 0, params)); } int index = 0; for (Pair>> nodeScanRangeMap : scanRange) { - params.instanceExecParams.get(index % params.instanceExecParams.size()) - .addBucketSeq(nodeScanRangeMap.first); + instanceExecParams.get(index % instanceExecParams.size()).addBucketSeq(nodeScanRangeMap.first); index++; } + params.instanceExecParams.addAll(instanceExecParams); } else { int expectedInstanceNum = 1; if (parallelExecInstanceNum > 1) { From d3714bf55619fffed08250687f614dcaea271458 Mon Sep 17 00:00:00 2001 From: meiyi Date: Tue, 26 Nov 2024 22:55:56 +0800 Subject: [PATCH 109/110] [fix](cloud) cloud mode support txn load for mow tables (#41932) --- .../cloud_engine_calc_delete_bitmap_task.cpp | 106 ++++++++-- .../cloud_engine_calc_delete_bitmap_task.h | 9 +- be/src/cloud/cloud_tablet.cpp | 6 +- be/src/cloud/cloud_tablet.h | 3 +- be/src/olap/base_tablet.cpp | 60 ++++-- be/src/olap/base_tablet.h | 15 +- be/src/olap/calc_delete_bitmap_executor.cpp | 6 +- be/src/olap/calc_delete_bitmap_executor.h | 3 +- be/src/olap/tablet.cpp | 2 +- be/src/olap/tablet.h | 3 +- be/src/olap/tablet_meta.h | 1 + be/src/olap/txn_manager.h | 6 + cloud/src/meta-service/meta_service_txn.cpp | 181 ++++++++---------- .../java/org/apache/doris/common/Config.java | 3 + .../CloudGlobalTransactionMgr.java | 161 +++++++++++----- .../doris/transaction/TransactionEntry.java | 9 - gensrc/thrift/AgentService.thrift | 1 + .../insert_p0/transaction/txn_insert.groovy | 11 +- .../txn_insert_concurrent_insert_mow.groovy | 135 +++++++++++++ ..._insert_concurrent_insert_aggregate.groovy | 2 +- ..._insert_concurrent_insert_duplicate.groovy | 2 +- .../txn_insert_concurrent_insert_mor.groovy | 2 +- .../txn_insert_concurrent_insert_mow.groovy | 13 +- .../txn_insert_concurrent_insert_ud.groovy | 7 +- ...txn_insert_concurrent_insert_update.groovy | 2 +- .../txn_insert_with_schema_change.groovy | 2 +- 26 files changed, 511 insertions(+), 240 deletions(-) create mode 100644 regression-test/suites/insert_p0/transaction/txn_insert_concurrent_insert_mow.groovy rename regression-test/suites/insert_p2/{ => transaction}/txn_insert_concurrent_insert_aggregate.groovy (99%) rename regression-test/suites/insert_p2/{ => transaction}/txn_insert_concurrent_insert_duplicate.groovy (99%) rename regression-test/suites/insert_p2/{ => transaction}/txn_insert_concurrent_insert_mor.groovy (99%) rename regression-test/suites/insert_p2/{ => transaction}/txn_insert_concurrent_insert_mow.groovy (95%) rename regression-test/suites/insert_p2/{ => transaction}/txn_insert_concurrent_insert_ud.groovy (98%) rename regression-test/suites/insert_p2/{ => transaction}/txn_insert_concurrent_insert_update.groovy (99%) rename regression-test/suites/insert_p2/{ => transaction}/txn_insert_with_schema_change.groovy (99%) diff --git a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp index 6abc3958650ef68..7391449b73f1b10 100644 --- a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp +++ b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp @@ -75,7 +75,7 @@ Status CloudEngineCalcDeleteBitmapTask::execute() { for (size_t i = 0; i < partition.tablet_ids.size(); i++) { auto tablet_id = partition.tablet_ids[i]; auto tablet_calc_delete_bitmap_ptr = std::make_shared( - _engine, this, tablet_id, transaction_id, version); + _engine, this, tablet_id, transaction_id, version, partition.sub_txn_ids); if (has_compaction_stats) { tablet_calc_delete_bitmap_ptr->set_compaction_stats( partition.base_compaction_cnts[i], partition.cumulative_compaction_cnts[i], @@ -107,12 +107,13 @@ Status CloudEngineCalcDeleteBitmapTask::execute() { CloudTabletCalcDeleteBitmapTask::CloudTabletCalcDeleteBitmapTask( CloudStorageEngine& engine, CloudEngineCalcDeleteBitmapTask* engine_task, int64_t tablet_id, - int64_t transaction_id, int64_t version) + int64_t transaction_id, int64_t version, const std::vector& sub_txn_ids) : _engine(engine), _engine_calc_delete_bitmap_task(engine_task), _tablet_id(tablet_id), _transaction_id(transaction_id), - _version(version) { + _version(version), + _sub_txn_ids(sub_txn_ids) { _mem_tracker = MemTrackerLimiter::create_shared( MemTrackerLimiter::Type::OTHER, fmt::format("CloudTabletCalcDeleteBitmapTask#_transaction_id={}", _transaction_id)); @@ -189,6 +190,60 @@ Status CloudTabletCalcDeleteBitmapTask::handle() const { return error_st; } + int64_t t3 = MonotonicMicros(); + Status status; + if (_sub_txn_ids.empty()) { + status = _handle_rowset(tablet, _version); + } else { + std::stringstream ss; + for (const auto& sub_txn_id : _sub_txn_ids) { + ss << sub_txn_id << ", "; + } + LOG(INFO) << "start calc delete bitmap for txn_id=" << _transaction_id << ", sub_txn_ids=[" + << ss.str() << "], table_id=" << tablet->table_id() + << ", partition_id=" << tablet->partition_id() << ", tablet_id=" << _tablet_id + << ", start_version=" << _version; + std::vector invisible_rowsets; + DeleteBitmapPtr tablet_delete_bitmap = + std::make_shared(tablet->tablet_meta()->delete_bitmap()); + for (int i = 0; i < _sub_txn_ids.size(); ++i) { + int64_t sub_txn_id = _sub_txn_ids[i]; + int64_t version = _version + i; + LOG(INFO) << "start calc delete bitmap for txn_id=" << _transaction_id + << ", sub_txn_id=" << sub_txn_id << ", table_id=" << tablet->table_id() + << ", partition_id=" << tablet->partition_id() << ", tablet_id=" << _tablet_id + << ", start_version=" << _version << ", cur_version=" << version; + status = _handle_rowset(tablet, version, sub_txn_id, &invisible_rowsets, + tablet_delete_bitmap); + if (!status.ok()) { + LOG(INFO) << "failed to calculate delete bitmap on tablet" + << ", table_id=" << tablet->table_id() + << ", transaction_id=" << _transaction_id << ", sub_txn_id=" << sub_txn_id + << ", tablet_id=" << tablet->tablet_id() << ", start version=" << _version + << ", cur_version=" << version << ", status=" << status; + return status; + } + DCHECK(invisible_rowsets.size() == i + 1); + } + } + auto total_update_delete_bitmap_time_us = MonotonicMicros() - t3; + LOG(INFO) << "calculate delete bitmap successfully on tablet" + << ", table_id=" << tablet->table_id() << ", transaction_id=" << _transaction_id + << ", tablet_id=" << tablet->tablet_id() + << ", get_tablet_time_us=" << get_tablet_time_us + << ", sync_rowset_time_us=" << sync_rowset_time_us + << ", total_update_delete_bitmap_time_us=" << total_update_delete_bitmap_time_us + << ", res=" << status; + return status; +} + +Status CloudTabletCalcDeleteBitmapTask::_handle_rowset( + std::shared_ptr tablet, int64_t version, int64_t sub_txn_id, + std::vector* invisible_rowsets, + DeleteBitmapPtr tablet_delete_bitmap) const { + int64_t transaction_id = sub_txn_id == -1 ? _transaction_id : sub_txn_id; + std::string txn_str = "txn_id=" + std::to_string(_transaction_id) + + (sub_txn_id == -1 ? "" : ", sub_txn_id=" + std::to_string(sub_txn_id)); RowsetSharedPtr rowset; DeleteBitmapPtr delete_bitmap; RowsetIdUnorderedSet rowset_ids; @@ -197,59 +252,76 @@ Status CloudTabletCalcDeleteBitmapTask::handle() const { int64_t txn_expiration; TxnPublishInfo previous_publish_info; Status status = _engine.txn_delete_bitmap_cache().get_tablet_txn_info( - _transaction_id, _tablet_id, &rowset, &delete_bitmap, &rowset_ids, &txn_expiration, + transaction_id, _tablet_id, &rowset, &delete_bitmap, &rowset_ids, &txn_expiration, &partial_update_info, &publish_status, &previous_publish_info); if (status != Status::OK()) { - LOG(WARNING) << "failed to get tablet txn info. tablet_id=" << _tablet_id - << ", txn_id=" << _transaction_id << ", status=" << status; + LOG(WARNING) << "failed to get tablet txn info. tablet_id=" << _tablet_id << ", " << txn_str + << ", status=" << status; _engine_calc_delete_bitmap_task->add_error_tablet_id(_tablet_id, status); return status; } int64_t t3 = MonotonicMicros(); - rowset->set_version(Version(_version, _version)); + rowset->set_version(Version(version, version)); TabletTxnInfo txn_info; txn_info.rowset = rowset; txn_info.delete_bitmap = delete_bitmap; txn_info.rowset_ids = rowset_ids; txn_info.partial_update_info = partial_update_info; txn_info.publish_status = publish_status; - txn_info.publish_info = {.publish_version = _version, + txn_info.publish_info = {.publish_version = version, .base_compaction_cnt = _ms_base_compaction_cnt, .cumulative_compaction_cnt = _ms_cumulative_compaction_cnt, .cumulative_point = _ms_cumulative_point}; auto update_delete_bitmap_time_us = 0; if (txn_info.publish_status && (*(txn_info.publish_status) == PublishStatus::SUCCEED) && - _version == previous_publish_info.publish_version && + version == previous_publish_info.publish_version && _ms_base_compaction_cnt == previous_publish_info.base_compaction_cnt && _ms_cumulative_compaction_cnt == previous_publish_info.cumulative_compaction_cnt && _ms_cumulative_point == previous_publish_info.cumulative_point) { // if version or compaction stats can't match, it means that this is a retry and there are // compaction or other loads finished successfully on the same tablet. So the previous publish // is stale and we should re-calculate the delete bitmap - LOG(INFO) << "tablet=" << _tablet_id << ",txn=" << _transaction_id + LOG(INFO) << "tablet=" << _tablet_id << ", " << txn_str << ",publish_status=SUCCEED,not need to recalculate and update delete_bitmap."; } else { - status = CloudTablet::update_delete_bitmap(tablet, &txn_info, _transaction_id, - txn_expiration); + if (invisible_rowsets == nullptr) { + status = CloudTablet::update_delete_bitmap(tablet, &txn_info, transaction_id, + txn_expiration); + } else { + txn_info.is_txn_load = true; + txn_info.invisible_rowsets = *invisible_rowsets; + txn_info.lock_id = _transaction_id; + txn_info.next_visible_version = _version; + status = CloudTablet::update_delete_bitmap(tablet, &txn_info, transaction_id, + txn_expiration, tablet_delete_bitmap); + } update_delete_bitmap_time_us = MonotonicMicros() - t3; } if (status != Status::OK()) { LOG(WARNING) << "failed to calculate delete bitmap. rowset_id=" << rowset->rowset_id() - << ", tablet_id=" << _tablet_id << ", txn_id=" << _transaction_id - << ", status=" << status; + << ", tablet_id=" << _tablet_id << ", " << txn_str << ", status=" << status; _engine_calc_delete_bitmap_task->add_error_tablet_id(_tablet_id, status); return status; } _engine_calc_delete_bitmap_task->add_succ_tablet_id(_tablet_id); LOG(INFO) << "calculate delete bitmap successfully on tablet" - << ", table_id=" << tablet->table_id() << ", transaction_id=" << _transaction_id + << ", table_id=" << tablet->table_id() << ", " << txn_str << ", tablet_id=" << tablet->tablet_id() << ", num_rows=" << rowset->num_rows() - << ", get_tablet_time_us=" << get_tablet_time_us - << ", sync_rowset_time_us=" << sync_rowset_time_us << ", update_delete_bitmap_time_us=" << update_delete_bitmap_time_us << ", res=" << status; + if (invisible_rowsets != nullptr) { + invisible_rowsets->push_back(rowset); + // see CloudTablet::save_delete_bitmap + auto dm = txn_info.delete_bitmap->delete_bitmap; + for (auto it = dm.begin(); it != dm.end(); ++it) { + if (std::get<1>(it->first) != DeleteBitmap::INVALID_SEGMENT_ID) { + tablet_delete_bitmap->merge( + {std::get<0>(it->first), std::get<1>(it->first), version}, it->second); + } + } + } return status; } diff --git a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.h b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.h index e3733d3e696ff82..c70a9cfa3903ba6 100644 --- a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.h +++ b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.h @@ -34,7 +34,8 @@ class CloudTabletCalcDeleteBitmapTask { public: CloudTabletCalcDeleteBitmapTask(CloudStorageEngine& engine, CloudEngineCalcDeleteBitmapTask* engine_task, int64_t tablet_id, - int64_t transaction_id, int64_t version); + int64_t transaction_id, int64_t version, + const std::vector& sub_txn_ids); ~CloudTabletCalcDeleteBitmapTask() = default; void set_compaction_stats(int64_t ms_base_compaction_cnt, int64_t ms_cumulative_compaction_cnt, @@ -43,12 +44,18 @@ class CloudTabletCalcDeleteBitmapTask { Status handle() const; private: + Status _handle_rowset(std::shared_ptr tablet, int64_t version, + int64_t sub_txn_id = -1, + std::vector* invisible_rowsets = nullptr, + DeleteBitmapPtr tablet_delete_bitmap = nullptr) const; + CloudStorageEngine& _engine; CloudEngineCalcDeleteBitmapTask* _engine_calc_delete_bitmap_task; int64_t _tablet_id; int64_t _transaction_id; int64_t _version; + std::vector _sub_txn_ids; int64_t _ms_base_compaction_cnt {-1}; int64_t _ms_cumulative_compaction_cnt {-1}; diff --git a/be/src/cloud/cloud_tablet.cpp b/be/src/cloud/cloud_tablet.cpp index c88b073e96494a8..267c204c0e608eb 100644 --- a/be/src/cloud/cloud_tablet.cpp +++ b/be/src/cloud/cloud_tablet.cpp @@ -690,7 +690,8 @@ CalcDeleteBitmapExecutor* CloudTablet::calc_delete_bitmap_executor() { Status CloudTablet::save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t txn_id, DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, - const RowsetIdUnorderedSet& cur_rowset_ids) { + const RowsetIdUnorderedSet& cur_rowset_ids, + int64_t lock_id) { RowsetSharedPtr rowset = txn_info->rowset; int64_t cur_version = rowset->start_version(); // update delete bitmap info, in order to avoid recalculation when trying again @@ -714,8 +715,9 @@ Status CloudTablet::save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t tx } } + auto ms_lock_id = lock_id == -1 ? txn_id : lock_id; RETURN_IF_ERROR(_engine.meta_mgr().update_delete_bitmap( - *this, txn_id, COMPACTION_DELETE_BITMAP_LOCK_ID, new_delete_bitmap.get())); + *this, ms_lock_id, COMPACTION_DELETE_BITMAP_LOCK_ID, new_delete_bitmap.get())); // store the delete bitmap with sentinel marks in txn_delete_bitmap_cache because if the txn is retried for some reason, // it will use the delete bitmap from txn_delete_bitmap_cache when re-calculating the delete bitmap, during which it will do diff --git a/be/src/cloud/cloud_tablet.h b/be/src/cloud/cloud_tablet.h index 0fde2f5b1d93ffa..80038e569ba2fc9 100644 --- a/be/src/cloud/cloud_tablet.h +++ b/be/src/cloud/cloud_tablet.h @@ -170,7 +170,8 @@ class CloudTablet final : public BaseTablet { Status save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t txn_id, DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, - const RowsetIdUnorderedSet& cur_rowset_ids) override; + const RowsetIdUnorderedSet& cur_rowset_ids, + int64_t lock_id = -1) override; Status calc_delete_bitmap_for_compaction(const std::vector& input_rowsets, const RowsetSharedPtr& output_rowset, diff --git a/be/src/olap/base_tablet.cpp b/be/src/olap/base_tablet.cpp index 2e70e4586cc7682..a499a27b07f6e2f 100644 --- a/be/src/olap/base_tablet.cpp +++ b/be/src/olap/base_tablet.cpp @@ -450,7 +450,8 @@ Status BaseTablet::lookup_row_key(const Slice& encoded_key, TabletSchema* latest RowLocation* row_location, uint32_t version, std::vector>& segment_caches, RowsetSharedPtr* rowset, bool with_rowid, - std::string* encoded_seq_value, OlapReaderStatistics* stats) { + std::string* encoded_seq_value, OlapReaderStatistics* stats, + DeleteBitmapPtr delete_bitmap) { SCOPED_BVAR_LATENCY(g_tablet_lookup_rowkey_latency); size_t seq_col_length = 0; // use the latest tablet schema to decide if the tablet has sequence column currently @@ -467,6 +468,8 @@ Status BaseTablet::lookup_row_key(const Slice& encoded_key, TabletSchema* latest Slice(encoded_key.get_data(), encoded_key.get_size() - seq_col_length - rowid_length); RowLocation loc; + auto tablet_delete_bitmap = + delete_bitmap == nullptr ? _tablet_meta->delete_bitmap_ptr() : delete_bitmap; for (size_t i = 0; i < specified_rowsets.size(); i++) { const auto& rs = specified_rowsets[i]; const auto& segments_key_bounds = rs->rowset_meta()->get_segments_key_bounds(); @@ -501,7 +504,7 @@ Status BaseTablet::lookup_row_key(const Slice& encoded_key, TabletSchema* latest if (!s.ok() && !s.is()) { return s; } - if (s.ok() && _tablet_meta->delete_bitmap().contains_agg_without_cache( + if (s.ok() && tablet_delete_bitmap->contains_agg_without_cache( {loc.rowset_id, loc.segment_id, version}, loc.row_id)) { // if has sequence col, we continue to compare the sequence_id of // all rowsets, util we find an existing key. @@ -535,7 +538,8 @@ Status BaseTablet::calc_delete_bitmap(const BaseTabletSPtr& tablet, RowsetShared const std::vector& segments, const std::vector& specified_rowsets, DeleteBitmapPtr delete_bitmap, int64_t end_version, - CalcDeleteBitmapToken* token, RowsetWriter* rowset_writer) { + CalcDeleteBitmapToken* token, RowsetWriter* rowset_writer, + DeleteBitmapPtr tablet_delete_bitmap) { auto rowset_id = rowset->rowset_id(); if (specified_rowsets.empty() || segments.empty()) { LOG(INFO) << "skip to construct delete bitmap tablet: " << tablet->tablet_id() @@ -548,10 +552,11 @@ Status BaseTablet::calc_delete_bitmap(const BaseTabletSPtr& tablet, RowsetShared const auto& seg = segment; if (token != nullptr) { RETURN_IF_ERROR(token->submit(tablet, rowset, seg, specified_rowsets, end_version, - delete_bitmap, rowset_writer)); + delete_bitmap, rowset_writer, tablet_delete_bitmap)); } else { RETURN_IF_ERROR(tablet->calc_segment_delete_bitmap( - rowset, segment, specified_rowsets, delete_bitmap, end_version, rowset_writer)); + rowset, segment, specified_rowsets, delete_bitmap, end_version, rowset_writer, + tablet_delete_bitmap)); } } @@ -562,7 +567,8 @@ Status BaseTablet::calc_segment_delete_bitmap(RowsetSharedPtr rowset, const segment_v2::SegmentSharedPtr& seg, const std::vector& specified_rowsets, DeleteBitmapPtr delete_bitmap, int64_t end_version, - RowsetWriter* rowset_writer) { + RowsetWriter* rowset_writer, + DeleteBitmapPtr tablet_delete_bitmap) { OlapStopWatch watch; auto rowset_id = rowset->rowset_id(); Version dummy_version(end_version + 1, end_version + 1); @@ -676,9 +682,16 @@ Status BaseTablet::calc_segment_delete_bitmap(RowsetSharedPtr rowset, } RowsetSharedPtr rowset_find; - auto st = lookup_row_key(key, rowset_schema.get(), true, specified_rowsets, &loc, - cast_set(dummy_version.first - 1), segment_caches, - &rowset_find); + Status st = Status::OK(); + if (tablet_delete_bitmap == nullptr) { + st = lookup_row_key(key, rowset_schema.get(), true, specified_rowsets, &loc, + cast_set(dummy_version.first - 1), segment_caches, + &rowset_find); + } else { + st = lookup_row_key(key, rowset_schema.get(), true, specified_rowsets, &loc, + cast_set(dummy_version.first - 1), segment_caches, + &rowset_find, true, nullptr, nullptr, tablet_delete_bitmap); + } bool expected_st = st.ok() || st.is() || st.is(); // It's a defensive DCHECK, we need to exclude some common errors to avoid core-dump // while stress test @@ -1351,7 +1364,8 @@ Status BaseTablet::check_delete_bitmap_correctness(DeleteBitmapPtr delete_bitmap } Status BaseTablet::update_delete_bitmap(const BaseTabletSPtr& self, TabletTxnInfo* txn_info, - int64_t txn_id, int64_t txn_expiration) { + int64_t txn_id, int64_t txn_expiration, + DeleteBitmapPtr tablet_delete_bitmap) { SCOPED_BVAR_LATENCY(g_tablet_update_delete_bitmap_latency); RowsetIdUnorderedSet cur_rowset_ids; RowsetIdUnorderedSet rowset_ids_to_add; @@ -1380,6 +1394,8 @@ Status BaseTablet::update_delete_bitmap(const BaseTabletSPtr& self, TabletTxnInf auto t1 = watch.get_elapse_time_us(); { + int64_t next_visible_version = txn_info->is_txn_load ? txn_info->next_visible_version + : txn_info->rowset->start_version(); std::shared_lock meta_rlock(self->_meta_lock); // tablet is under alter process. The delete bitmap will be calculated after conversion. if (self->tablet_state() == TABLET_NOTREADY) { @@ -1387,7 +1403,7 @@ Status BaseTablet::update_delete_bitmap(const BaseTabletSPtr& self, TabletTxnInf << self->tablet_id(); return Status::OK(); } - RETURN_IF_ERROR(self->get_all_rs_id_unlocked(cur_version - 1, &cur_rowset_ids)); + RETURN_IF_ERROR(self->get_all_rs_id_unlocked(next_visible_version - 1, &cur_rowset_ids)); } auto t2 = watch.get_elapse_time_us(); @@ -1402,6 +1418,15 @@ Status BaseTablet::update_delete_bitmap(const BaseTabletSPtr& self, TabletTxnInf std::shared_lock meta_rlock(self->_meta_lock); specified_rowsets = self->get_rowset_by_ids(&rowset_ids_to_add); } + if (txn_info->is_txn_load) { + for (auto invisible_rowset : txn_info->invisible_rowsets) { + specified_rowsets.emplace_back(invisible_rowset); + } + std::sort(specified_rowsets.begin(), specified_rowsets.end(), + [](RowsetSharedPtr& lhs, RowsetSharedPtr& rhs) { + return lhs->end_version() > rhs->end_version(); + }); + } auto t3 = watch.get_elapse_time_us(); // If a rowset is produced by compaction before the commit phase of the partial update load @@ -1446,7 +1471,8 @@ Status BaseTablet::update_delete_bitmap(const BaseTabletSPtr& self, TabletTxnInf auto token = self->calc_delete_bitmap_executor()->create_token(); // set rowset_writer to nullptr to skip the alignment process RETURN_IF_ERROR(calc_delete_bitmap(self, rowset, segments, rowsets_skip_alignment, - delete_bitmap, cur_version - 1, token.get(), nullptr)); + delete_bitmap, cur_version - 1, token.get(), nullptr, + tablet_delete_bitmap)); RETURN_IF_ERROR(token->wait()); } @@ -1454,13 +1480,14 @@ Status BaseTablet::update_delete_bitmap(const BaseTabletSPtr& self, TabletTxnInf // Otherwise, it will be submitted to the thread pool for calculation. if (segments.size() <= 1) { RETURN_IF_ERROR(calc_delete_bitmap(self, rowset, segments, specified_rowsets, delete_bitmap, - cur_version - 1, nullptr, transient_rs_writer.get())); + cur_version - 1, nullptr, transient_rs_writer.get(), + tablet_delete_bitmap)); } else { auto token = self->calc_delete_bitmap_executor()->create_token(); RETURN_IF_ERROR(calc_delete_bitmap(self, rowset, segments, specified_rowsets, delete_bitmap, - cur_version - 1, token.get(), - transient_rs_writer.get())); + cur_version - 1, token.get(), transient_rs_writer.get(), + tablet_delete_bitmap)); RETURN_IF_ERROR(token->wait()); } @@ -1511,8 +1538,9 @@ Status BaseTablet::update_delete_bitmap(const BaseTabletSPtr& self, TabletTxnInf segments.begin(), segments.end(), 0, [](size_t sum, const segment_v2::SegmentSharedPtr& s) { return sum += s->num_rows(); }); auto t5 = watch.get_elapse_time_us(); + int64_t lock_id = txn_info->is_txn_load ? txn_info->lock_id : -1; RETURN_IF_ERROR(self->save_delete_bitmap(txn_info, txn_id, delete_bitmap, - transient_rs_writer.get(), cur_rowset_ids)); + transient_rs_writer.get(), cur_rowset_ids, lock_id)); LOG(INFO) << "[Publish] construct delete bitmap tablet: " << self->tablet_id() << ", rowset_ids to add: " << rowset_ids_to_add.size() << ", rowset_ids to del: " << rowset_ids_to_del.size() diff --git a/be/src/olap/base_tablet.h b/be/src/olap/base_tablet.h index b6fc953e46082ed..f961f4c49eedd66 100644 --- a/be/src/olap/base_tablet.h +++ b/be/src/olap/base_tablet.h @@ -156,7 +156,8 @@ class BaseTablet { std::vector>& segment_caches, RowsetSharedPtr* rowset = nullptr, bool with_rowid = true, std::string* encoded_seq_value = nullptr, - OlapReaderStatistics* stats = nullptr); + OlapReaderStatistics* stats = nullptr, + DeleteBitmapPtr tablet_delete_bitmap = nullptr); // calc delete bitmap when flush memtable, use a fake version to calc // For example, cur max version is 5, and we use version 6 to calc but @@ -169,13 +170,15 @@ class BaseTablet { const std::vector& specified_rowsets, DeleteBitmapPtr delete_bitmap, int64_t version, CalcDeleteBitmapToken* token, - RowsetWriter* rowset_writer = nullptr); + RowsetWriter* rowset_writer = nullptr, + DeleteBitmapPtr tablet_delete_bitmap = nullptr); Status calc_segment_delete_bitmap(RowsetSharedPtr rowset, const segment_v2::SegmentSharedPtr& seg, const std::vector& specified_rowsets, DeleteBitmapPtr delete_bitmap, int64_t end_version, - RowsetWriter* rowset_writer); + RowsetWriter* rowset_writer, + DeleteBitmapPtr tablet_delete_bitmap = nullptr); Status calc_delete_bitmap_between_segments( RowsetSharedPtr rowset, const std::vector& segments, @@ -235,11 +238,13 @@ class BaseTablet { int64_t txn_expiration = 0) = 0; static Status update_delete_bitmap(const BaseTabletSPtr& self, TabletTxnInfo* txn_info, - int64_t txn_id, int64_t txn_expiration = 0); + int64_t txn_id, int64_t txn_expiration = 0, + DeleteBitmapPtr tablet_delete_bitmap = nullptr); virtual Status save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t txn_id, DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, - const RowsetIdUnorderedSet& cur_rowset_ids) = 0; + const RowsetIdUnorderedSet& cur_rowset_ids, + int64_t lock_id = -1) = 0; virtual CalcDeleteBitmapExecutor* calc_delete_bitmap_executor() = 0; void calc_compaction_output_rowset_delete_bitmap( diff --git a/be/src/olap/calc_delete_bitmap_executor.cpp b/be/src/olap/calc_delete_bitmap_executor.cpp index 3983dc0a98642a3..e45f9801f68ba4c 100644 --- a/be/src/olap/calc_delete_bitmap_executor.cpp +++ b/be/src/olap/calc_delete_bitmap_executor.cpp @@ -34,7 +34,8 @@ Status CalcDeleteBitmapToken::submit(BaseTabletSPtr tablet, RowsetSharedPtr cur_ const segment_v2::SegmentSharedPtr& cur_segment, const std::vector& target_rowsets, int64_t end_version, DeleteBitmapPtr delete_bitmap, - RowsetWriter* rowset_writer) { + RowsetWriter* rowset_writer, + DeleteBitmapPtr tablet_delete_bitmap) { { std::shared_lock rlock(_lock); RETURN_IF_ERROR(_status); @@ -44,7 +45,8 @@ Status CalcDeleteBitmapToken::submit(BaseTabletSPtr tablet, RowsetSharedPtr cur_ return _thread_token->submit_func([=, this]() { SCOPED_ATTACH_TASK(_query_thread_context); auto st = tablet->calc_segment_delete_bitmap(cur_rowset, cur_segment, target_rowsets, - delete_bitmap, end_version, rowset_writer); + delete_bitmap, end_version, rowset_writer, + tablet_delete_bitmap); if (!st.ok()) { LOG(WARNING) << "failed to calc segment delete bitmap, tablet_id: " << tablet->tablet_id() << " rowset: " << cur_rowset->rowset_id() diff --git a/be/src/olap/calc_delete_bitmap_executor.h b/be/src/olap/calc_delete_bitmap_executor.h index fa1e79b7feaa197..288108b04971df8 100644 --- a/be/src/olap/calc_delete_bitmap_executor.h +++ b/be/src/olap/calc_delete_bitmap_executor.h @@ -52,7 +52,8 @@ class CalcDeleteBitmapToken { Status submit(BaseTabletSPtr tablet, RowsetSharedPtr cur_rowset, const segment_v2::SegmentSharedPtr& cur_segment, const std::vector& target_rowsets, int64_t end_version, - DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer); + DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, + DeleteBitmapPtr tablet_delete_bitmap); // wait all tasks in token to be completed. Status wait(); diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 8a0e23e75b86adb..0d04984d0e06bae 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -2490,7 +2490,7 @@ CalcDeleteBitmapExecutor* Tablet::calc_delete_bitmap_executor() { Status Tablet::save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t txn_id, DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, - const RowsetIdUnorderedSet& cur_rowset_ids) { + const RowsetIdUnorderedSet& cur_rowset_ids, int64_t lock_id) { RowsetSharedPtr rowset = txn_info->rowset; int64_t cur_version = rowset->start_version(); diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index f5866c67641581c..0b7d758ab8fd880 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -417,7 +417,8 @@ class Tablet final : public BaseTablet { CalcDeleteBitmapExecutor* calc_delete_bitmap_executor() override; Status save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t txn_id, DeleteBitmapPtr delete_bitmap, RowsetWriter* rowset_writer, - const RowsetIdUnorderedSet& cur_rowset_ids) override; + const RowsetIdUnorderedSet& cur_rowset_ids, + int64_t lock_id = -1) override; void merge_delete_bitmap(const DeleteBitmap& delete_bitmap); bool check_all_rowset_segment(); diff --git a/be/src/olap/tablet_meta.h b/be/src/olap/tablet_meta.h index d56e529e42bf4b4..fb0895604a19fe0 100644 --- a/be/src/olap/tablet_meta.h +++ b/be/src/olap/tablet_meta.h @@ -236,6 +236,7 @@ class TabletMeta : public MetadataAdder { static void init_column_from_tcolumn(uint32_t unique_id, const TColumn& tcolumn, ColumnPB* column); + DeleteBitmapPtr delete_bitmap_ptr() { return _delete_bitmap; } DeleteBitmap& delete_bitmap() { return *_delete_bitmap; } bool enable_unique_key_merge_on_write() const { return _enable_unique_key_merge_on_write; } diff --git a/be/src/olap/txn_manager.h b/be/src/olap/txn_manager.h index 88ee97c5f6a3b90..1994dec949414f0 100644 --- a/be/src/olap/txn_manager.h +++ b/be/src/olap/txn_manager.h @@ -87,6 +87,12 @@ struct TabletTxnInfo { std::shared_ptr publish_status; TxnPublishInfo publish_info; + // for cloud only, used to calculate delete bitmap for txn load + bool is_txn_load = false; + std::vector invisible_rowsets; + int64_t lock_id; + int64_t next_visible_version; + TxnState state {TxnState::PREPARED}; TabletTxnInfo() = default; diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index 32f6b56f51af4cd..58930f6edfc63ba 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -912,6 +912,69 @@ void update_tablet_stats(const StatsTabletKeyInfo& info, const TabletStats& stat } } +// process mow table, check lock and remove pending key +void process_mow_when_commit_txn( + const CommitTxnRequest* request, const std::string& instance_id, MetaServiceCode& code, + std::string& msg, std::unique_ptr& txn, + std::unordered_map>& table_id_tablet_ids) { + int64_t txn_id = request->txn_id(); + std::stringstream ss; + std::vector lock_keys; + lock_keys.reserve(request->mow_table_ids().size()); + for (auto table_id : request->mow_table_ids()) { + lock_keys.push_back(meta_delete_bitmap_update_lock_key({instance_id, table_id, -1})); + } + std::vector> lock_values; + TxnErrorCode err = txn->batch_get(&lock_values, lock_keys); + if (err != TxnErrorCode::TXN_OK) { + ss << "failed to get delete bitmap update lock key info, instance_id=" << instance_id + << " err=" << err; + msg = ss.str(); + code = cast_as(err); + LOG(WARNING) << msg << " txn_id=" << txn_id; + return; + } + size_t total_locks = lock_keys.size(); + for (size_t i = 0; i < total_locks; i++) { + int64_t table_id = request->mow_table_ids(i); + // When the key does not exist, it means the lock has been acquired + // by another transaction and successfully committed. + if (!lock_values[i].has_value()) { + ss << "get delete bitmap update lock info, lock is expired" + << " table_id=" << table_id << " key=" << hex(lock_keys[i]); + code = MetaServiceCode::LOCK_EXPIRED; + msg = ss.str(); + LOG(WARNING) << msg << " txn_id=" << txn_id; + return; + } + + DeleteBitmapUpdateLockPB lock_info; + if (!lock_info.ParseFromString(lock_values[i].value())) [[unlikely]] { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse DeleteBitmapUpdateLockPB"; + LOG(WARNING) << msg << " txn_id=" << txn_id; + return; + } + if (lock_info.lock_id() != request->txn_id()) { + msg = "lock is expired"; + code = MetaServiceCode::LOCK_EXPIRED; + return; + } + txn->remove(lock_keys[i]); + LOG(INFO) << "xxx remove delete bitmap lock, lock_key=" << hex(lock_keys[i]) + << " txn_id=" << txn_id; + + for (auto tablet_id : table_id_tablet_ids[table_id]) { + std::string pending_key = meta_pending_delete_bitmap_key({instance_id, tablet_id}); + txn->remove(pending_key); + LOG(INFO) << "xxx remove delete bitmap pending key, pending_key=" << hex(pending_key) + << " txn_id=" << txn_id; + } + } + lock_keys.clear(); + lock_values.clear(); +} + /** * 0. Extract txn_id from request * 1. Get db id from TxnKv with txn_id @@ -1173,61 +1236,11 @@ void commit_txn_immediately( stats.num_segs += i.num_segments(); } // for tmp_rowsets_meta - // process mow table, check lock and remove pending key - std::vector lock_keys; - lock_keys.reserve(request->mow_table_ids().size()); - for (auto table_id : request->mow_table_ids()) { - lock_keys.push_back(meta_delete_bitmap_update_lock_key({instance_id, table_id, -1})); - } - std::vector> lock_values; - err = txn->batch_get(&lock_values, lock_keys); - if (err != TxnErrorCode::TXN_OK) { - ss << "failed to get delete bitmap update lock key info, instance_id=" << instance_id - << " err=" << err; - msg = ss.str(); - code = cast_as(err); - LOG(WARNING) << msg << " txn_id=" << txn_id; + process_mow_when_commit_txn(request, instance_id, code, msg, txn, table_id_tablet_ids); + if (code != MetaServiceCode::OK) { + LOG(WARNING) << "process mow failed, txn_id=" << txn_id << " code=" << code; return; } - size_t total_locks = lock_keys.size(); - for (size_t i = 0; i < total_locks; i++) { - int64_t table_id = request->mow_table_ids(i); - // When the key does not exist, it means the lock has been acquired - // by another transaction and successfully committed. - if (!lock_values[i].has_value()) { - ss << "get delete bitmap update lock info, lock is expired" - << " table_id=" << table_id << " key=" << hex(lock_keys[i]); - code = MetaServiceCode::LOCK_EXPIRED; - msg = ss.str(); - LOG(WARNING) << msg << " txn_id=" << txn_id; - return; - } - - DeleteBitmapUpdateLockPB lock_info; - if (!lock_info.ParseFromString(lock_values[i].value())) [[unlikely]] { - code = MetaServiceCode::PROTOBUF_PARSE_ERR; - msg = "failed to parse DeleteBitmapUpdateLockPB"; - LOG(WARNING) << msg << " txn_id=" << txn_id; - return; - } - if (lock_info.lock_id() != request->txn_id()) { - msg = "lock is expired"; - code = MetaServiceCode::LOCK_EXPIRED; - return; - } - txn->remove(lock_keys[i]); - LOG(INFO) << "xxx remove delete bitmap lock, lock_key=" << hex(lock_keys[i]) - << " txn_id=" << txn_id; - - for (auto tablet_id : table_id_tablet_ids[table_id]) { - std::string pending_key = meta_pending_delete_bitmap_key({instance_id, tablet_id}); - txn->remove(pending_key); - LOG(INFO) << "xxx remove delete bitmap pending key, pending_key=" - << hex(pending_key) << " txn_id=" << txn_id; - } - } - lock_keys.clear(); - lock_values.clear(); // Save rowset meta for (auto& i : rowsets) { @@ -1810,62 +1823,12 @@ void commit_txn_eventually( response->add_versions(i.second + 1); } - // process mow table, check lock and remove pending key - std::vector lock_keys; - lock_keys.reserve(request->mow_table_ids().size()); - for (auto table_id : request->mow_table_ids()) { - lock_keys.push_back(meta_delete_bitmap_update_lock_key({instance_id, table_id, -1})); - } - std::vector> lock_values; - err = txn->batch_get(&lock_values, lock_keys); - if (err != TxnErrorCode::TXN_OK) { - ss << "failed to get delete bitmap update lock key info, instance_id=" << instance_id - << " err=" << err; - msg = ss.str(); - code = cast_as(err); - LOG(WARNING) << msg << " txn_id=" << txn_id; + process_mow_when_commit_txn(request, instance_id, code, msg, txn, table_id_tablet_ids); + if (code != MetaServiceCode::OK) { + LOG(WARNING) << "process mow failed, txn_id=" << txn_id << " code=" << code; return; } - for (size_t i = 0; i < lock_keys.size(); i++) { - int64_t table_id = request->mow_table_ids(i); - // When the key does not exist, it means the lock has been acquired - // by another transaction and successfully committed. - if (!lock_values[i].has_value()) { - ss << "get delete bitmap update lock info, lock is expired" - << " table_id=" << table_id << " key=" << hex(lock_keys[i]); - code = MetaServiceCode::LOCK_EXPIRED; - msg = ss.str(); - LOG(WARNING) << msg << " txn_id=" << txn_id; - return; - } - - DeleteBitmapUpdateLockPB lock_info; - if (!lock_info.ParseFromString(lock_values[i].value())) [[unlikely]] { - code = MetaServiceCode::PROTOBUF_PARSE_ERR; - msg = "failed to parse DeleteBitmapUpdateLockPB"; - LOG(WARNING) << msg << " txn_id=" << txn_id; - return; - } - if (lock_info.lock_id() != request->txn_id()) { - msg = "lock is expired"; - code = MetaServiceCode::LOCK_EXPIRED; - return; - } - txn->remove(lock_keys[i]); - LOG(INFO) << "xxx remove delete bitmap lock, lock_key=" << hex(lock_keys[i]) - << " txn_id=" << txn_id; - - for (auto tablet_id : table_id_tablet_ids[table_id]) { - std::string pending_key = meta_pending_delete_bitmap_key({instance_id, tablet_id}); - txn->remove(pending_key); - LOG(INFO) << "xxx remove delete bitmap pending key, pending_key=" - << hex(pending_key) << " txn_id=" << txn_id; - } - } - lock_keys.clear(); - lock_values.clear(); - // Save table versions for (auto& i : table_id_tablet_ids) { std::string ver_key = table_version_key({instance_id, db_id, i.first}); @@ -2282,6 +2245,12 @@ void commit_txn_with_sub_txn(const CommitTxnRequest* request, CommitTxnResponse* } // for tmp_rowsets_meta } + process_mow_when_commit_txn(request, instance_id, code, msg, txn, table_id_tablet_ids); + if (code != MetaServiceCode::OK) { + LOG(WARNING) << "process mow failed, txn_id=" << txn_id << " code=" << code; + return; + } + // Save rowset meta for (auto& i : rowsets) { size_t rowset_size = i.first.size() + i.second.size(); diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index be0390db584ca92..72e6438999c3961 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -3188,6 +3188,9 @@ public static int metaServiceRpcRetryTimes() { @ConfField(mutable = true, description = {"存算分离模式下calculate delete bitmap task 超时时间,默认15s"}) public static int calculate_delete_bitmap_task_timeout_seconds = 15; + @ConfField(mutable = true, description = {"存算分离模式下事务导入calculate delete bitmap task 超时时间,默认300s"}) + public static int calculate_delete_bitmap_task_timeout_seconds_for_transaction_load = 300; + @ConfField(mutable = true, description = {"存算分离模式下commit阶段等锁超时时间,默认5s"}) public static int try_commit_lock_timeout_seconds = 5; diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java index 1a80058759bba92..3c56f7dc56a1800 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java @@ -99,6 +99,7 @@ import org.apache.doris.thrift.TCalcDeleteBitmapPartitionInfo; import org.apache.doris.thrift.TStatus; import org.apache.doris.thrift.TStatusCode; +import org.apache.doris.thrift.TTabletCommitInfo; import org.apache.doris.thrift.TTaskType; import org.apache.doris.thrift.TUniqueId; import org.apache.doris.thrift.TWaitingTxnStatusRequest; @@ -492,7 +493,7 @@ private void commitTransaction(long dbId, List
tableList, long transactio transactionState.getTransactionStatus().toString()); } } - calcDeleteBitmapForMow(dbId, mowTableList, transactionId, tabletCommitInfos); + calcDeleteBitmapForMow(dbId, mowTableList, transactionId, tabletCommitInfos, null); } CommitTxnRequest.Builder builder = CommitTxnRequest.newBuilder(); @@ -529,12 +530,17 @@ private void commitTransaction(long dbId, List
tableList, long transactio } final CommitTxnRequest commitTxnRequest = builder.build(); + executeCommitTxnRequest(commitTxnRequest, transactionId, is2PC, txnCommitAttachment); + } + + private void executeCommitTxnRequest(CommitTxnRequest commitTxnRequest, long transactionId, boolean is2PC, + TxnCommitAttachment txnCommitAttachment) throws UserException { boolean txnOperated = false; TransactionState txnState = null; TxnStateChangeCallback cb = null; long callbackId = 0L; try { - txnState = commitTxn(commitTxnRequest, transactionId, is2PC, dbId, tableList); + txnState = commitTxn(commitTxnRequest, transactionId, is2PC); txnOperated = true; if (DebugPointUtil.isEnable("CloudGlobalTransactionMgr.commitTransaction.timeout")) { throw new UserException(InternalErrorCode.DELETE_BITMAP_LOCK_ERR, @@ -558,8 +564,8 @@ private void commitTransaction(long dbId, List
tableList, long transactio } } - private TransactionState commitTxn(CommitTxnRequest commitTxnRequest, long transactionId, boolean is2PC, long dbId, - List
tableList) throws UserException { + private TransactionState commitTxn(CommitTxnRequest commitTxnRequest, long transactionId, boolean is2PC) + throws UserException { CommitTxnResponse commitTxnResponse = null; TransactionState txnState = null; int retryTime = 0; @@ -648,9 +654,9 @@ private List getMowTableList(List
tableList, List tableList, long transactionId, - List tabletCommitInfos) + List tabletCommitInfos, List subTransactionStates) throws UserException { - Map>> backendToPartitionTablets = Maps.newHashMap(); + Map>> backendToPartitionTablets = Maps.newHashMap(); Map partitions = Maps.newHashMap(); Map> tableToPartitions = Maps.newHashMap(); Map> tableToTabletList = Maps.newHashMap(); @@ -661,6 +667,8 @@ private void calcDeleteBitmapForMow(long dbId, List tableList, long t throw new UserException("The partition info is empty, table may be dropped, txnid=" + transactionId); } + Map> partitionToSubTxnIds = getPartitionSubTxnIds(subTransactionStates, tableToTabletList, + tabletToTabletMeta); Map baseCompactionCnts = Maps.newHashMap(); Map cumulativeCompactionCnts = Maps.newHashMap(); Map cumulativePoints = Maps.newHashMap(); @@ -670,9 +678,11 @@ private void calcDeleteBitmapForMow(long dbId, List tableList, long t Map> backendToPartitionInfos = getCalcDeleteBitmapInfo( backendToPartitionTablets, partitionVersions, baseCompactionCnts, cumulativeCompactionCnts, - cumulativePoints); + cumulativePoints, partitionToSubTxnIds); try { - sendCalcDeleteBitmaptask(dbId, transactionId, backendToPartitionInfos); + sendCalcDeleteBitmaptask(dbId, transactionId, backendToPartitionInfos, + subTransactionStates == null ? Config.calculate_delete_bitmap_task_timeout_seconds + : Config.calculate_delete_bitmap_task_timeout_seconds_for_transaction_load); } catch (UserException e) { LOG.warn("failed to sendCalcDeleteBitmaptask for txn=" + transactionId + ",exception=" + e.getMessage()); removeDeleteBitmapUpdateLock(tableToPartitions, transactionId); @@ -680,11 +690,34 @@ private void calcDeleteBitmapForMow(long dbId, List tableList, long t } } + private Map> getPartitionSubTxnIds(List subTransactionStates, + Map> tableToTabletList, Map tabletToTabletMeta) { + if (subTransactionStates == null) { + return null; + } + Map> partitionToSubTxnIds = Maps.newHashMap(); + for (SubTransactionState subTransactionState : subTransactionStates) { + if (!tableToTabletList.containsKey(subTransactionState.getTable().getId())) { + // skip non mow table + continue; + } + for (TTabletCommitInfo ci : subTransactionState.getTabletCommitInfos()) { + TabletMeta tabletMeta = tabletToTabletMeta.get(ci.getTabletId()); + long partitionId = tabletMeta.getPartitionId(); + List subTxnIds = partitionToSubTxnIds.computeIfAbsent(partitionId, k -> Lists.newArrayList()); + if (!subTxnIds.contains(subTransactionState.getSubTransactionId())) { + subTxnIds.add(subTransactionState.getSubTransactionId()); + } + } + } + return partitionToSubTxnIds; + } + private void getPartitionInfo(List tableList, List tabletCommitInfos, Map> tableToParttions, Map partitions, - Map>> backendToPartitionTablets, + Map>> backendToPartitionTablets, Map> tableToTabletList, Map tabletToTabletMeta) { Map tableMap = Maps.newHashMap(); @@ -697,18 +730,22 @@ private void getPartitionInfo(List tableList, TabletInvertedIndex tabletInvertedIndex = Env.getCurrentEnv().getTabletInvertedIndex(); List tabletMetaList = tabletInvertedIndex.getTabletMetaList(tabletIds); for (int i = 0; i < tabletMetaList.size(); i++) { + long tabletId = tabletIds.get(i); + if (tabletToTabletMeta.containsKey(tabletId)) { + continue; + } TabletMeta tabletMeta = tabletMetaList.get(i); long tableId = tabletMeta.getTableId(); if (!tableMap.containsKey(tableId)) { continue; } - tabletToTabletMeta.put(tabletIds.get(i), tabletMeta); + tabletToTabletMeta.put(tabletId, tabletMeta); - if (!tableToTabletList.containsKey(tableId)) { - tableToTabletList.put(tableId, Lists.newArrayList()); + List tableTabletIds = tableToTabletList.computeIfAbsent(tableId, k -> Lists.newArrayList()); + if (!tableTabletIds.contains(tabletId)) { + tableTabletIds.add(tabletId); } - tableToTabletList.get(tableId).add(tabletIds.get(i)); long partitionId = tabletMeta.getPartitionId(); long backendId = tabletCommitInfos.get(i).getBackendId(); @@ -721,11 +758,11 @@ private void getPartitionInfo(List tableList, if (!backendToPartitionTablets.containsKey(backendId)) { backendToPartitionTablets.put(backendId, Maps.newHashMap()); } - Map> partitionToTablets = backendToPartitionTablets.get(backendId); + Map> partitionToTablets = backendToPartitionTablets.get(backendId); if (!partitionToTablets.containsKey(partitionId)) { - partitionToTablets.put(partitionId, Lists.newArrayList()); + partitionToTablets.put(partitionId, Sets.newHashSet()); } - partitionToTablets.get(partitionId).add(tabletIds.get(i)); + partitionToTablets.get(partitionId).add(tabletId); partitions.putIfAbsent(partitionId, tableMap.get(tableId).getPartition(partitionId)); } } @@ -741,18 +778,18 @@ private Map getPartitionVersions(Map partitionMap) } private Map> getCalcDeleteBitmapInfo( - Map>> backendToPartitionTablets, Map partitionVersions, + Map>> backendToPartitionTablets, Map partitionVersions, Map baseCompactionCnts, Map cumulativeCompactionCnts, - Map cumulativePoints) { + Map cumulativePoints, Map> partitionToSubTxnIds) { Map> backendToPartitionInfos = Maps.newHashMap(); - for (Map.Entry>> entry : backendToPartitionTablets.entrySet()) { + for (Map.Entry>> entry : backendToPartitionTablets.entrySet()) { List partitionInfos = Lists.newArrayList(); - for (Map.Entry> partitionToTablets : entry.getValue().entrySet()) { + for (Map.Entry> partitionToTablets : entry.getValue().entrySet()) { Long partitionId = partitionToTablets.getKey(); - List tabletList = partitionToTablets.getValue(); + Set tabletList = partitionToTablets.getValue(); TCalcDeleteBitmapPartitionInfo partitionInfo = new TCalcDeleteBitmapPartitionInfo(partitionId, partitionVersions.get(partitionId), - tabletList); + Lists.newArrayList(tabletList)); if (!baseCompactionCnts.isEmpty() && !cumulativeCompactionCnts.isEmpty() && !cumulativePoints.isEmpty()) { List reqBaseCompactionCnts = Lists.newArrayList(); @@ -766,6 +803,13 @@ private Map> getCalcDeleteBitmapInfo( partitionInfo.setBaseCompactionCnts(reqBaseCompactionCnts); partitionInfo.setCumulativeCompactionCnts(reqCumulativeCompactionCnts); partitionInfo.setCumulativePoints(reqCumulativePoints); + if (partitionToSubTxnIds != null) { + List subTxnIds = partitionToSubTxnIds.get(partitionId); + if (subTxnIds != null && !subTxnIds.isEmpty()) { + partitionInfo.setSubTxnIds(subTxnIds); + LOG.debug("partitionId: {}, subTxnIds: {}", partitionId, subTxnIds); + } + } } partitionInfos.add(partitionInfo); } @@ -926,8 +970,8 @@ private void removeDeleteBitmapUpdateLock(Map> tableToParttions, } private void sendCalcDeleteBitmaptask(long dbId, long transactionId, - Map> backendToPartitionInfos) - throws UserException { + Map> backendToPartitionInfos, + long calculateDeleteBitmapTaskTimeoutSeconds) throws UserException { if (backendToPartitionInfos.isEmpty()) { return; } @@ -948,13 +992,14 @@ private void sendCalcDeleteBitmaptask(long dbId, long transactionId, // not check return value, because the add will success AgentTaskQueue.addTask(task); batchTask.addTask(task); - LOG.info("send calculate delete bitmap task to be {}, txn_id {}", entry.getKey(), transactionId); + LOG.info("send calculate delete bitmap task to be {}, txn_id {}, partitionInfos={}", entry.getKey(), + transactionId, entry.getValue()); } AgentTaskExecutor.submit(batchTask); boolean ok; try { - ok = countDownLatch.await(Config.calculate_delete_bitmap_task_timeout_seconds, TimeUnit.SECONDS); + ok = countDownLatch.await(calculateDeleteBitmapTaskTimeoutSeconds, TimeUnit.SECONDS); } catch (InterruptedException e) { LOG.warn("InterruptedException: ", e); ok = false; @@ -1043,14 +1088,42 @@ public boolean commitAndPublishTransaction(DatabaseIf db, long transactionId, "disable_load_job is set to true, all load jobs are not allowed"); } LOG.info("try to commit transaction, txnId: {}, subTxnStates: {}", transactionId, subTransactionStates); + + Preconditions.checkState(db instanceof Database); + List tableIdList = subTransactionStates.stream().map(s -> s.getTable().getId()).distinct() + .collect(Collectors.toList()); + List
tableList = ((Database) db).getTablesOnIdOrderOrThrowException(tableIdList); + beforeCommitTransaction(tableList, transactionId, timeoutMillis); + try { + commitTransactionWithSubTxns(db.getId(), tableList, transactionId, subTransactionStates); + } finally { + decreaseWaitingLockCount(tableList); + MetaLockUtils.commitUnlockTables(tableList); + } + return true; + } + + private void commitTransactionWithSubTxns(long dbId, List
tableList, long transactionId, + List subTransactionStates) throws UserException { + List tabletCommitInfos = subTransactionStates.stream().map( + SubTransactionState::getTabletCommitInfos).flatMap(Collection::stream) + .map(c -> new TabletCommitInfo(c.getTabletId(), c.getBackendId())).collect(Collectors.toList()); + List mowTableList = getMowTableList(tableList, tabletCommitInfos); + if (!mowTableList.isEmpty()) { + calcDeleteBitmapForMow(dbId, mowTableList, transactionId, tabletCommitInfos, subTransactionStates); + } + cleanSubTransactions(transactionId); CommitTxnRequest.Builder builder = CommitTxnRequest.newBuilder(); - builder.setDbId(db.getId()) + builder.setDbId(dbId) .setTxnId(transactionId) .setIs2Pc(false) .setCloudUniqueId(Config.cloud_unique_id) .setIsTxnLoad(true) .setEnableTxnLazyCommit(Config.enable_cloud_txn_lazy_commit); + for (OlapTable olapTable : mowTableList) { + builder.addMowTableIds(olapTable.getId()); + } // add sub txn infos for (SubTransactionState subTransactionState : subTransactionStates) { builder.addSubTxnInfos(SubTxnInfo.newBuilder().setSubTxnId(subTransactionState.getSubTransactionId()) @@ -1064,31 +1137,12 @@ public boolean commitAndPublishTransaction(DatabaseIf db, long transactionId, } final CommitTxnRequest commitTxnRequest = builder.build(); - TransactionState txnState = null; - boolean txnOperated = false; - try { - txnState = commitTxn(commitTxnRequest, transactionId, false, db.getId(), - subTransactionStates.stream().map(SubTransactionState::getTable) - .collect(Collectors.toList())); - txnOperated = true; - } finally { - if (txnState != null) { - TxnStateChangeCallback cb = callbackFactory.getCallback(txnState.getCallbackId()); - if (cb != null) { - LOG.info("commitTxn, run txn callback, transactionId:{} callbackId:{}, txnState:{}", - txnState.getTransactionId(), txnState.getCallbackId(), txnState); - cb.afterCommitted(txnState, txnOperated); - cb.afterVisible(txnState, txnOperated); - } - } - } - return true; + executeCommitTxnRequest(commitTxnRequest, transactionId, false, null); } - @Override - public boolean commitAndPublishTransaction(DatabaseIf db, List
tableList, long transactionId, - List tabletCommitInfos, long timeoutMillis, - TxnCommitAttachment txnCommitAttachment) throws UserException { + // add some log and get commit lock, mainly used for mow tables + private void beforeCommitTransaction(List
tableList, long transactionId, long timeoutMillis) + throws UserException { for (int i = 0; i < tableList.size(); i++) { long tableId = tableList.get(i).getId(); LOG.info("start commit txn=" + transactionId + ",table=" + tableId); @@ -1107,6 +1161,13 @@ public boolean commitAndPublishTransaction(DatabaseIf db, List
tableList, "get table cloud commit lock timeout, tableList=(" + StringUtils.join(tableList, ",") + ")"); } + } + + @Override + public boolean commitAndPublishTransaction(DatabaseIf db, List
tableList, long transactionId, + List tabletCommitInfos, long timeoutMillis, + TxnCommitAttachment txnCommitAttachment) throws UserException { + beforeCommitTransaction(tableList, transactionId, timeoutMillis); try { commitTransaction(db.getId(), tableList, transactionId, tabletCommitInfos, txnCommitAttachment); } finally { diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionEntry.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionEntry.java index 25c4ff4b3b24933..8a75dd0d0c2303b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionEntry.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionEntry.java @@ -21,8 +21,6 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; -import org.apache.doris.catalog.KeysType; -import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.TableIf; import org.apache.doris.cloud.transaction.CloudGlobalTransactionMgr; @@ -206,13 +204,6 @@ public long beginTransaction(TableIf table, SubTransactionType subTransactionTyp throw new AnalysisException( "Transaction insert can not insert into values and insert into select at the same time"); } - if (Config.isCloudMode()) { - OlapTable olapTable = (OlapTable) table; - if (olapTable.getKeysType() == KeysType.UNIQUE_KEYS && olapTable.getEnableUniqueKeyMergeOnWrite()) { - throw new UserException( - "Transaction load is not supported for merge on write unique keys table in cloud mode"); - } - } DatabaseIf database = table.getDatabase(); if (!isTransactionBegan) { long timeoutSecond = ConnectContext.get().getExecTimeout(); diff --git a/gensrc/thrift/AgentService.thrift b/gensrc/thrift/AgentService.thrift index abffd176ef8ea76..fdbf4483bf8c4cf 100644 --- a/gensrc/thrift/AgentService.thrift +++ b/gensrc/thrift/AgentService.thrift @@ -440,6 +440,7 @@ struct TCalcDeleteBitmapPartitionInfo { 4: optional list base_compaction_cnts 5: optional list cumulative_compaction_cnts 6: optional list cumulative_points + 7: optional list sub_txn_ids } struct TCalcDeleteBitmapRequest { diff --git a/regression-test/suites/insert_p0/transaction/txn_insert.groovy b/regression-test/suites/insert_p0/transaction/txn_insert.groovy index 6653c05740e3b19..a4868ca6b11ffe2 100644 --- a/regression-test/suites/insert_p0/transaction/txn_insert.groovy +++ b/regression-test/suites/insert_p0/transaction/txn_insert.groovy @@ -584,19 +584,10 @@ suite("txn_insert") { } catch (Exception e) { logger.info("exception: " + e) sql """ rollback """ - if (isCloudMode()) { - assertTrue(e.getMessage().contains("Transaction load is not supported for merge on write unique keys table in cloud mode")) - } else { - assertTrue(false, "should not reach here") - } + assertTrue(false, "should not reach here") } } - // the following cases are not supported in cloud mode - if (isCloudMode()) { - break - } - // 16. update stmt(mow table) if (use_nereids_planner) { def ut_table = "txn_insert_ut" diff --git a/regression-test/suites/insert_p0/transaction/txn_insert_concurrent_insert_mow.groovy b/regression-test/suites/insert_p0/transaction/txn_insert_concurrent_insert_mow.groovy new file mode 100644 index 000000000000000..8e6de4dd9e9a383 --- /dev/null +++ b/regression-test/suites/insert_p0/transaction/txn_insert_concurrent_insert_mow.groovy @@ -0,0 +1,135 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import java.sql.Connection +import java.sql.DriverManager +import java.sql.Statement +import java.util.concurrent.TimeUnit +import java.util.concurrent.CompletableFuture + +suite("txn_insert_concurrent_insert_mow") { + def tableName = "txn_insert_concurrent_insert_mow" + List errors = new ArrayList<>() + + for (int i = 0; i < 3; i++) { + def table_name = "${tableName}_${i}" + sql """ drop table if exists ${table_name} """ + sql """ + CREATE TABLE IF NOT EXISTS ${table_name} ( + L_ORDERKEY INTEGER NOT NULL, + L_PARTKEY INTEGER NOT NULL, + L_SUPPKEY INTEGER NOT NULL, + L_LINENUMBER INTEGER NOT NULL, + L_QUANTITY DECIMAL(15,2) NOT NULL, + L_EXTENDEDPRICE DECIMAL(15,2) NOT NULL, + L_DISCOUNT DECIMAL(15,2) NOT NULL, + L_TAX DECIMAL(15,2) NOT NULL, + L_RETURNFLAG CHAR(1) NOT NULL, + L_LINESTATUS CHAR(1) NOT NULL, + L_SHIPDATE DATE NOT NULL, + L_COMMITDATE DATE NOT NULL, + L_RECEIPTDATE DATE NOT NULL, + L_SHIPINSTRUCT CHAR(25) NOT NULL, + L_SHIPMODE CHAR(10) NOT NULL, + L_COMMENT VARCHAR(44) NOT NULL + ) + UNIQUE KEY(L_ORDERKEY, L_PARTKEY, L_SUPPKEY, L_LINENUMBER) + DISTRIBUTED BY HASH(L_ORDERKEY) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + if (i == 0) { + continue + } + + streamLoad { + table table_name + set 'column_separator', '|' + set 'compress_type', 'GZ' + set 'columns', "l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag,l_linestatus, l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment,temp" + file """${getS3Url()}/regression/tpch/sf0.1/lineitem.tbl.gz""" + + time 10000 // limit inflight 10s + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + assertEquals(json.NumberTotalRows, json.NumberLoadedRows) + assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) + } + } + } + sql """ sync """ + + def dbName = "regression_test_insert_p0_transaction" + def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, dbName).replace("&useServerPrepStmts=true", "") + "&useLocalSessionState=true" + logger.info("url: ${url}") + + def sqls = [ + "begin", + "insert into ${tableName}_0 select * from ${tableName}_1 where L_ORDERKEY < 30000;", + "insert into ${tableName}_1 select * from ${tableName}_2 where L_ORDERKEY > 500000;", + "insert into ${tableName}_0 select * from ${tableName}_2 where L_ORDERKEY < 30000;", + "commit" + ] + def txn_insert = { -> + try (Connection conn = DriverManager.getConnection(url, context.config.jdbcUser, context.config.jdbcPassword); + Statement stmt = conn.createStatement()) { + for (def sql : sqls) { + logger.info(Thread.currentThread().getName() + " execute sql: " + sql) + stmt.execute(sql) + } + logger.info("finish txn insert for " + Thread.currentThread().getName()) + } catch (Throwable e) { + logger.error("txn insert failed", e) + errors.add("txn insert failed " + e.getMessage()) + } + } + + List> futures = new ArrayList<>() + for (int i = 0; i < 10; i++) { + CompletableFuture future = CompletableFuture.runAsync(txn_insert) + futures.add(future) + } + CompletableFuture[] futuresArray = futures.toArray(new CompletableFuture[0]) + CompletableFuture.allOf(futuresArray).get(10, TimeUnit.MINUTES) + sql """ sync """ + + logger.info("error num: " + errors.size() + ", errors: " + errors) + + def result = sql """ select count() from ${tableName}_0 """ + logger.info("result: ${result}") + assertEquals(30209, result[0][0]) + result = sql """ select count() from ${tableName}_1 """ + logger.info("result: ${result}") + assertEquals(600572, result[0][0]) + + def db_name = "regression_test_insert_p0_transaction" + def tables = sql """ show tables from $db_name """ + logger.info("tables: $tables") + for (def table_info : tables) { + def table_name = table_info[0] + if (table_name.startsWith(tableName)) { + check_table_version_continuous(db_name, table_name) + } + } +} diff --git a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_aggregate.groovy b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_aggregate.groovy similarity index 99% rename from regression-test/suites/insert_p2/txn_insert_concurrent_insert_aggregate.groovy rename to regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_aggregate.groovy index 1fbe18bf212c478..fa57c5501187bc6 100644 --- a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_aggregate.groovy +++ b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_aggregate.groovy @@ -81,7 +81,7 @@ suite("txn_insert_concurrent_insert_aggregate") { } sql """ sync """ - def dbName = "regression_test_insert_p2" + def dbName = "regression_test_insert_p2_transaction" def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, dbName).replace("&useServerPrepStmts=true", "") + "&useLocalSessionState=true" logger.info("url: ${url}") diff --git a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_duplicate.groovy b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_duplicate.groovy similarity index 99% rename from regression-test/suites/insert_p2/txn_insert_concurrent_insert_duplicate.groovy rename to regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_duplicate.groovy index 048a07fb817fff6..e771078f1fbecbd 100644 --- a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_duplicate.groovy +++ b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_duplicate.groovy @@ -81,7 +81,7 @@ suite("txn_insert_concurrent_insert_duplicate") { } sql """ sync """ - def dbName = "regression_test_insert_p2" + def dbName = "regression_test_insert_p2_transaction" def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, dbName).replace("&useServerPrepStmts=true", "") + "&useLocalSessionState=true" logger.info("url: ${url}") diff --git a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_mor.groovy b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_mor.groovy similarity index 99% rename from regression-test/suites/insert_p2/txn_insert_concurrent_insert_mor.groovy rename to regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_mor.groovy index c67119328ef7a61..418992835d0551c 100644 --- a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_mor.groovy +++ b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_mor.groovy @@ -82,7 +82,7 @@ suite("txn_insert_concurrent_insert_mor") { } sql """ sync """ - def dbName = "regression_test_insert_p2" + def dbName = "regression_test_insert_p2_transaction" def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, dbName).replace("&useServerPrepStmts=true", "") + "&useLocalSessionState=true" logger.info("url: ${url}") diff --git a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_mow.groovy b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_mow.groovy similarity index 95% rename from regression-test/suites/insert_p2/txn_insert_concurrent_insert_mow.groovy rename to regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_mow.groovy index f8a971db75ee2f9..4d6e297cac918b7 100644 --- a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_mow.groovy +++ b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_mow.groovy @@ -22,11 +22,6 @@ import java.util.concurrent.TimeUnit import java.util.concurrent.CompletableFuture suite("txn_insert_concurrent_insert_mow") { - if (isCloudMode()) { - logger.info("cloud txn load does not support mow") - return - } - def tableName = "txn_insert_concurrent_insert_mow" List errors = new ArrayList<>() @@ -85,8 +80,12 @@ suite("txn_insert_concurrent_insert_mow") { } } sql """ sync """ + def t2_row_count = 6001215 + def result = sql """ select count() from ${tableName}_2 """ + logger.info("${tableName}_2 row count: ${result}, expected: ${t2_row_count}") + assertEquals(t2_row_count, result[0][0] as int) - def dbName = "regression_test_insert_p2" + def dbName = "regression_test_insert_p2_transaction" def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, dbName).replace("&useServerPrepStmts=true", "") + "&useLocalSessionState=true" logger.info("url: ${url}") @@ -156,7 +155,7 @@ suite("txn_insert_concurrent_insert_mow") { logger.info("error num: " + errors.size() + ", errors: " + errors) def t0_row_count = 6001215 - def result = sql """ select count() from ${tableName}_0 """ + result = sql """ select count() from ${tableName}_0 """ logger.info("${tableName}_0 row count: ${result}, expected: ${t0_row_count}") def t1_row_count = 2999666 diff --git a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_ud.groovy b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_ud.groovy similarity index 98% rename from regression-test/suites/insert_p2/txn_insert_concurrent_insert_ud.groovy rename to regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_ud.groovy index a524703f9ef99fb..d0b27641c64a99f 100644 --- a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_ud.groovy +++ b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_ud.groovy @@ -23,11 +23,6 @@ import java.util.concurrent.CompletableFuture // test update and delete command suite("txn_insert_concurrent_insert_ud") { - if (isCloudMode()) { - logger.info("cloud txn load does not support mow") - return - } - def tableName = "txn_insert_concurrent_insert_ud" List errors = new ArrayList<>() @@ -88,7 +83,7 @@ suite("txn_insert_concurrent_insert_ud") { } sql """ sync """ - def dbName = "regression_test_insert_p2" + def dbName = "regression_test_insert_p2_transaction" def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, dbName).replace("&useServerPrepStmts=true", "") + "&useLocalSessionState=true" logger.info("url: ${url}") diff --git a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_update.groovy b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_update.groovy similarity index 99% rename from regression-test/suites/insert_p2/txn_insert_concurrent_insert_update.groovy rename to regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_update.groovy index b467a87de8201c8..eba69918660cafb 100644 --- a/regression-test/suites/insert_p2/txn_insert_concurrent_insert_update.groovy +++ b/regression-test/suites/insert_p2/transaction/txn_insert_concurrent_insert_update.groovy @@ -84,7 +84,7 @@ suite("txn_insert_concurrent_insert_update") { } sql """ sync """ - def dbName = "regression_test_insert_p2" + def dbName = "regression_test_insert_p2_transaction" def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, dbName).replace("&useServerPrepStmts=true", "") + "&useLocalSessionState=true" logger.info("url: ${url}") diff --git a/regression-test/suites/insert_p2/txn_insert_with_schema_change.groovy b/regression-test/suites/insert_p2/transaction/txn_insert_with_schema_change.groovy similarity index 99% rename from regression-test/suites/insert_p2/txn_insert_with_schema_change.groovy rename to regression-test/suites/insert_p2/transaction/txn_insert_with_schema_change.groovy index 56692b68d3730dc..34b859b5e80d0ae 100644 --- a/regression-test/suites/insert_p2/txn_insert_with_schema_change.groovy +++ b/regression-test/suites/insert_p2/transaction/txn_insert_with_schema_change.groovy @@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit // schema change and modify replica num suite("txn_insert_with_schema_change") { def tableName = "txn_insert_with_schema_change" - def dbName = "regression_test_insert_p2" + def dbName = "regression_test_insert_p2_transaction" def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, dbName).replace("&useServerPrepStmts=true", "") + "&useLocalSessionState=true" logger.info("url: ${url}") From 267a8cc9ea9e25cf66d7e1b0be645cf75e5100ce Mon Sep 17 00:00:00 2001 From: meiyi Date: Tue, 26 Nov 2024 23:08:44 +0800 Subject: [PATCH 110/110] [fix](cluster key) add cluster key case (#44488) --- .../java/org/apache/doris/catalog/Table.java | 5 +- .../translator/PhysicalPlanTranslator.java | 2 +- .../data/nereids_p0/delete/delete_cte_ck.out | 29 +++++ .../nereids_p0/delete/delete_cte_ck.groovy | 123 ++++++++++++++++++ 4 files changed, 156 insertions(+), 3 deletions(-) create mode 100644 regression-test/data/nereids_p0/delete/delete_cte_ck.out create mode 100644 regression-test/suites/nereids_p0/delete/delete_cte_ck.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java index d85d98a8ea550f5..ed37a936e108196 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java @@ -40,6 +40,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -393,7 +394,7 @@ public TableType getType() { } public List getFullSchema() { - return fullSchema; + return ImmutableList.copyOf(fullSchema); } // should override in subclass if necessary @@ -403,7 +404,7 @@ public List getBaseSchema() { public List getBaseSchema(boolean full) { if (full) { - return fullSchema; + return ImmutableList.copyOf(fullSchema); } else { return fullSchema.stream().filter(Column::isVisible).collect(Collectors.toList()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 654ccc8ca1155aa..56ae65ec7229413 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -2695,7 +2695,7 @@ private boolean checkPushSort(SortNode sortNode, OlapTable olapTable) { if (sortExprs.size() > olapTable.getDataSortInfo().getColNum()) { return false; } - List sortKeyColumns = olapTable.getFullSchema(); + List sortKeyColumns = new ArrayList<>(olapTable.getFullSchema()); if (olapTable.getEnableUniqueKeyMergeOnWrite()) { Map clusterKeyMap = new TreeMap<>(); for (Column column : olapTable.getFullSchema()) { diff --git a/regression-test/data/nereids_p0/delete/delete_cte_ck.out b/regression-test/data/nereids_p0/delete/delete_cte_ck.out new file mode 100644 index 000000000000000..2734e5b4a33b33e --- /dev/null +++ b/regression-test/data/nereids_p0/delete/delete_cte_ck.out @@ -0,0 +1,29 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +1 \N 2 1 1.0 \N +1 10 1 1 1.0 2000-01-01 +2 \N 4 2 2.0 \N +2 20 2 2 2.0 2000-01-02 +3 \N 6 3 3.0 \N +3 30 3 3 3.0 2000-01-03 + +-- !sql -- +2 \N 4 2 2.0 \N +2 20 2 2 2.0 2000-01-02 +3 \N 6 3 3.0 \N +3 30 3 3 3.0 2000-01-03 + +-- !sql -- +1 \N 2 1 1.0 \N +1 10 1 1 1.0 2000-01-01 +2 \N 4 2 2.0 \N +2 20 2 2 2.0 2000-01-02 +3 \N 6 3 3.0 \N +3 30 3 3 3.0 2000-01-03 + +-- !sql -- +2 \N 4 2 2.0 \N +2 20 2 2 2.0 2000-01-02 +3 \N 6 3 3.0 \N +3 30 3 3 3.0 2000-01-03 + diff --git a/regression-test/suites/nereids_p0/delete/delete_cte_ck.groovy b/regression-test/suites/nereids_p0/delete/delete_cte_ck.groovy new file mode 100644 index 000000000000000..2e9bd1f8fb3fa99 --- /dev/null +++ b/regression-test/suites/nereids_p0/delete/delete_cte_ck.groovy @@ -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. + +suite('nereids_delete_cte_ck') { + + String db = context.config.getDbNameByFile(context.file) + sql "select 1;" // to create database + + for (def use_row_store : [false, true]) { + logger.info("current params: use_row_store: ${use_row_store}") + + connect(user = context.config.jdbcUser, password = context.config.jdbcPassword, url = context.config.jdbcUrl) { + sql "use ${db};" + + def t1 = 't1_cte_ck' + def t2 = 't2_cte_ck' + def t3 = 't3_cte_ck' + + sql "drop table if exists ${t1}" + sql """ + create table ${t1} ( + id int, + id1 int, + c1 bigint, + c2 string, + c3 double, + c4 date + ) unique key (id, id1) + CLUSTER BY (`id1`, `id`, `c4`) + distributed by hash(id, id1) + properties( + "replication_num"="1", + "enable_unique_key_merge_on_write" = "true", + "store_row_column" = "${use_row_store}"); """ + + sql "drop table if exists ${t2}" + sql """ + create table ${t2} ( + id int, + c1 bigint, + c2 string, + c3 double, + c4 date + ) unique key (id) + CLUSTER BY (`c4`, `c1`) + distributed by hash(id) + properties( + "replication_num"="1" + ); + """ + + sql "drop table if exists ${t3}" + sql """ + create table ${t3} ( + id int + ) distributed by hash(id) + properties( + "replication_num"="1" + ); + """ + + sql """ + INSERT INTO ${t1} VALUES + (1, 10, 1, '1', 1.0, '2000-01-01'), + (2, 20, 2, '2', 2.0, '2000-01-02'), + (3, 30, 3, '3', 3.0, '2000-01-03'); + """ + + sql """ + + INSERT INTO ${t2} VALUES + (1, 10, '10', 10.0, '2000-01-10'), + (2, 20, '20', 20.0, '2000-01-20'), + (3, 30, '30', 30.0, '2000-01-30'), + (4, 4, '4', 4.0, '2000-01-04'), + (5, 5, '5', 5.0, '2000-01-05'); + """ + + sql """ + INSERT INTO ${t3} VALUES + (1), + (4), + (5); + """ + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_nereids_dml=true" + + sql "insert into ${t1}(id, c1, c2, c3) select id, c1 * 2, c2, c3 from ${t1}" + sql "insert into ${t2}(id, c1, c2, c3) select id, c1, c2 * 2, c3 from ${t2}" + sql "insert into ${t2}(c1, c3) select c1 + 1, c3 + 1 from (select id, c1, c3 from ${t1} order by id, c1 limit 10) ${t1}, ${t3}" + + qt_sql "select * from ${t1} order by id, id1" + + sql """ + with cte as (select * from ${t3}) + delete from ${t1} + using ${t2} join cte on ${t2}.id = cte.id + where ${t1}.id = ${t2}.id; + """ + + qt_sql "select * from ${t1} order by id, id1" + + // test cte != null and using == null + sql """with cte as (select 1) delete from ${t1} where ${t1}.id = 100""" + } + } +} \ No newline at end of file