From d183e08f6d9dcfacf804362e3ab9ad78dfaa2b06 Mon Sep 17 00:00:00 2001 From: Ashin Gau Date: Wed, 6 Sep 2023 22:45:31 +0800 Subject: [PATCH 01/14] [opt](MergedIO) optimize merge small IO, prevent amplified read (#23849) There were two vulnerabilities in the previous fix(https://github.com/apache/doris/pull/20305): 1. `next_content` may not necessarily be a truly readable range 2. The last range of the merged data may be the hollow This PR fundamentally solves the problem of reading amplification by rechecking the calculation range. According to the algorithm, there is only one possibility of generating read amplification, with only a small content of data within the 4k(`MIN_READ_SIZE `) range. However, 4k is generally the minimum IO size and there is no need for further segmentation. --- be/src/io/fs/buffered_reader.cpp | 49 ++++++++++++++++++-------- be/src/io/fs/buffered_reader.h | 21 ++++++----- be/test/io/fs/buffered_reader_test.cpp | 45 +++++++++++++++++++++++ 3 files changed, 92 insertions(+), 23 deletions(-) diff --git a/be/src/io/fs/buffered_reader.cpp b/be/src/io/fs/buffered_reader.cpp index 00f88c75154b269..2a7187cc28fd675 100644 --- a/be/src/io/fs/buffered_reader.cpp +++ b/be/src/io/fs/buffered_reader.cpp @@ -49,7 +49,7 @@ Status MergeRangeFileReader::read_at_impl(size_t offset, Slice result, size_t* b if (result.size == 0) { return Status::OK(); } - int range_index = _search_read_range(offset, offset + result.size); + const int range_index = _search_read_range(offset, offset + result.size); if (range_index < 0) { SCOPED_RAW_TIMER(&_statistics.read_time); Status st = _reader->read_at(offset, result, bytes_read, io_ctx); @@ -99,6 +99,8 @@ Status MergeRangeFileReader::read_at_impl(size_t offset, Slice result, size_t* b // merge small IO size_t merge_start = offset + has_read; const size_t merge_end = merge_start + READ_SLICE_SIZE; + // + std::vector> merged_slice; size_t content_size = 0; size_t hollow_size = 0; if (merge_start > _random_access_ranges[range_index].end_offset) { @@ -118,12 +120,14 @@ Status MergeRangeFileReader::read_at_impl(size_t offset, Slice result, size_t* b size_t add_content = std::min(merge_end - merge_start, content_max); content_size += add_content; merge_start += add_content; + merged_slice.emplace_back(add_content, true); break; } size_t add_content = std::min(_random_access_ranges[merge_index].end_offset - merge_start, content_max); content_size += add_content; merge_start += add_content; + merged_slice.emplace_back(add_content, true); if (merge_start != _random_access_ranges[merge_index].end_offset) { break; } @@ -136,18 +140,9 @@ Status MergeRangeFileReader::read_at_impl(size_t offset, Slice result, size_t* b } if (gap < merge_end - merge_start && content_size < _remaining && !_range_cached_data[merge_index + 1].has_read) { - size_t next_content = - std::min(_random_access_ranges[merge_index + 1].end_offset, merge_end) - - _random_access_ranges[merge_index + 1].start_offset; - next_content = std::min(next_content, _remaining - content_size); - double amplified_ratio = config::max_amplified_read_ratio; - if ((content_size + hollow_size) > MIN_READ_SIZE && - (hollow_size + gap) > (next_content + content_size) * amplified_ratio) { - // too large gap - break; - } hollow_size += gap; merge_start = _random_access_ranges[merge_index + 1].start_offset; + merged_slice.emplace_back(gap, false); } else { // there's no enough memory to read hollow data break; @@ -155,7 +150,33 @@ Status MergeRangeFileReader::read_at_impl(size_t offset, Slice result, size_t* b } merge_index++; } - if (content_size + hollow_size == to_read) { + content_size = 0; + hollow_size = 0; + double amplified_ratio = config::max_amplified_read_ratio; + std::vector> ratio_and_size; + // Calculate the read amplified ratio for each merge operation and the size of the merged data. + // Find the largest size of the merged data whose amplified ratio is less than config::max_amplified_read_ratio + for (const std::pair& slice : merged_slice) { + if (slice.second) { + content_size += slice.first; + if (slice.first > 0) { + ratio_and_size.emplace_back((double)hollow_size / content_size, + content_size + hollow_size); + } + } else { + hollow_size += slice.first; + } + } + size_t best_merged_size = 0; + for (const std::pair& rs : ratio_and_size) { + if (rs.second > best_merged_size) { + if (rs.first < amplified_ratio || rs.second <= MIN_READ_SIZE) { + best_merged_size = rs.second; + } + } + } + + if (best_merged_size == to_read) { // read directly to avoid copy operation SCOPED_RAW_TIMER(&_statistics.read_time); size_t read_size = 0; @@ -170,8 +191,8 @@ Status MergeRangeFileReader::read_at_impl(size_t offset, Slice result, size_t* b merge_start = offset + has_read; size_t merge_read_size = 0; - RETURN_IF_ERROR(_fill_box(range_index, merge_start, content_size + hollow_size, - &merge_read_size, io_ctx)); + RETURN_IF_ERROR( + _fill_box(range_index, merge_start, best_merged_size, &merge_read_size, io_ctx)); if (cached_data.start_offset != merge_start) { return Status::IOError("Wrong start offset in merged IO"); } diff --git a/be/src/io/fs/buffered_reader.h b/be/src/io/fs/buffered_reader.h index 25a681133037f3c..84235f0a460b937 100644 --- a/be/src/io/fs/buffered_reader.h +++ b/be/src/io/fs/buffered_reader.h @@ -75,6 +75,15 @@ struct PrefetchRange { */ class MergeRangeFileReader : public io::FileReader { public: + struct Statistics { + int64_t copy_time = 0; + int64_t read_time = 0; + int64_t request_io = 0; + int64_t merged_io = 0; + int64_t request_bytes = 0; + int64_t read_bytes = 0; + }; + struct RangeCachedData { size_t start_offset; size_t end_offset; @@ -190,20 +199,14 @@ class MergeRangeFileReader : public io::FileReader { // for test only const std::vector& box_reference() const { return _box_ref; } + // for test only + const Statistics& statistics() const { return _statistics; } + protected: Status read_at_impl(size_t offset, Slice result, size_t* bytes_read, const IOContext* io_ctx) override; private: - struct Statistics { - int64_t copy_time = 0; - int64_t read_time = 0; - int64_t request_io = 0; - int64_t merged_io = 0; - int64_t request_bytes = 0; - int64_t read_bytes = 0; - }; - RuntimeProfile::Counter* _copy_time; RuntimeProfile::Counter* _read_time; RuntimeProfile::Counter* _request_io; diff --git a/be/test/io/fs/buffered_reader_test.cpp b/be/test/io/fs/buffered_reader_test.cpp index 6a281e125fc6a20..97ef2171366388d 100644 --- a/be/test/io/fs/buffered_reader_test.cpp +++ b/be/test/io/fs/buffered_reader_test.cpp @@ -270,6 +270,51 @@ TEST_F(BufferedReaderTest, test_miss) { EXPECT_EQ(45, bytes_read); } +TEST_F(BufferedReaderTest, test_read_amplify) { + size_t kb = 1024; + io::FileReaderSPtr offset_reader = std::make_shared(2048 * kb); // 2MB + std::vector random_access_ranges; + random_access_ranges.emplace_back(0, 1 * kb); // column0 + // if read the follow slice, amplified_ratio = 1, but data size <= MIN_READ_SIZE + random_access_ranges.emplace_back(3 * kb, 4 * kb); // column1 + // if read the follow slice, amplified_ratio = 1, + // but merge the next rand, amplified_ratio will be decreased + random_access_ranges.emplace_back(5 * kb, 6 * kb); // column2 + random_access_ranges.emplace_back(7 * kb, 12 * kb); // column3 + // read the last range first, so we can't merge the last range when reading the former ranges, + // even if the amplified_ratio < 0.8 + random_access_ranges.emplace_back(512 * kb, 2048 * kb); // column4 + + io::MergeRangeFileReader merge_reader(nullptr, offset_reader, random_access_ranges); + char data[2048 * kb]; // 2MB + Slice result(data, 2048 * kb); + size_t bytes_read = 0; + + // read column4 + result.size = 1024 * kb; + merge_reader.read_at(1024 * kb, result, &bytes_read, nullptr); + EXPECT_EQ(bytes_read, 1024 * kb); + EXPECT_EQ(merge_reader.statistics().request_bytes, 1024 * kb); + EXPECT_EQ(merge_reader.statistics().read_bytes, 1024 * kb); + // read column0 + result.size = 1 * kb; + // will merge column 0 ~ 3 + merge_reader.read_at(0, result, &bytes_read, nullptr); + EXPECT_EQ(bytes_read, 1 * kb); + EXPECT_EQ(merge_reader.statistics().read_bytes, 1024 * kb + 12 * kb); + // read column1 + result.size = 1 * kb; + merge_reader.read_at(3 * kb, result, &bytes_read, nullptr); + // read column2 + result.size = 1 * kb; + merge_reader.read_at(5 * kb, result, &bytes_read, nullptr); + // read column3 + result.size = 5 * kb; + merge_reader.read_at(7 * kb, result, &bytes_read, nullptr); + EXPECT_EQ(merge_reader.statistics().request_bytes, 1024 * kb + 8 * kb); + EXPECT_EQ(merge_reader.statistics().read_bytes, 1024 * kb + 12 * kb); +} + TEST_F(BufferedReaderTest, test_merged_io) { io::FileReaderSPtr offset_reader = std::make_shared(128 * 1024 * 1024); // 128MB From a532a089445a01cdb1b9d6c6845cbedba05be72b Mon Sep 17 00:00:00 2001 From: zhangdong <493738387@qq.com> Date: Wed, 6 Sep 2023 22:48:15 +0800 Subject: [PATCH 02/14] [Feature](Nereids)support insert overwrite for Nereids (#23682) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - support insert overwrite for Nereids - Solving the problem of table ID changing(When no partition is specified,automatically specify all partitions for table ) todo: - Under the current processing logic, if the master restarts midway, it will generate dirty data (temp partition) - for old planner ,insert overwrite table ,table id will change --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../doris/analysis/PartitionKeyDesc.java | 9 + .../org/apache/doris/catalog/OlapTable.java | 13 +- .../doris/datasource/InternalCatalog.java | 33 ++-- .../analyzer/UnboundOlapTableSink.java | 4 + .../nereids/parser/LogicalPlanBuilder.java | 3 +- .../trees/plans/commands/DeleteCommand.java | 2 +- .../commands/InsertIntoTableCommand.java | 181 +++++++++++++++++- .../trees/plans/commands/UpdateCommand.java | 2 +- .../apache/doris/planner/OlapTableSink.java | 7 +- .../insert_overwrite_table_list_nereids.out | 47 +++++ .../insert_overwrite_table_range_nereids.out | 47 +++++ ..._overwrite_table_unpartitioned_nereids.out | 22 +++ ...insert_overwrite_table_list_nereids.groovy | 166 ++++++++++++++++ ...nsert_overwrite_table_range_nereids.groovy | 166 ++++++++++++++++ ...erwrite_table_unpartitioned_nereids.groovy | 129 +++++++++++++ 16 files changed, 805 insertions(+), 28 deletions(-) create mode 100644 regression-test/data/insert_overwrite_p0/insert_overwrite_table_list_nereids.out create mode 100644 regression-test/data/insert_overwrite_p0/insert_overwrite_table_range_nereids.out create mode 100644 regression-test/data/insert_overwrite_p0/insert_overwrite_table_unpartitioned_nereids.out create mode 100644 regression-test/suites/insert_overwrite_p0/insert_overwrite_table_list_nereids.groovy create mode 100644 regression-test/suites/insert_overwrite_p0/insert_overwrite_table_range_nereids.groovy create mode 100644 regression-test/suites/insert_overwrite_p0/insert_overwrite_table_unpartitioned_nereids.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 b4bf8d2b3ba4e1a..88d1e4f4d5e5464 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 @@ -40,7 +40,7 @@ statement AS type=(RESTRICTIVE | PERMISSIVE) TO (user=userIdentify | ROLE roleName=identifier) USING LEFT_PAREN booleanExpression RIGHT_PAREN #createRowPolicy - | explain? INSERT INTO tableName=multipartIdentifier + | explain? INSERT (INTO | OVERWRITE TABLE) tableName=multipartIdentifier (PARTITION partition=identifierList)? // partition define (WITH LABEL labelName=identifier)? cols=identifierList? // label and columns define (LEFT_BRACKET hints=identifierSeq RIGHT_BRACKET)? // hint define diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionKeyDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionKeyDesc.java index a3e32c6745882ce..f7f63490ee8429a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionKeyDesc.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionKeyDesc.java @@ -28,6 +28,8 @@ // Describe the partition key values in create table or add partition clause public class PartitionKeyDesc { public static final PartitionKeyDesc MAX_VALUE = new PartitionKeyDesc(); + // UNPARTITIONED table not have real keyDesc,therefore, provide this DUMMY_KEY_DESC + public static final PartitionKeyDesc DUMMY_KEY_DESC = new PartitionKeyDesc(); public enum PartitionKeyValueType { INVALID, @@ -125,6 +127,10 @@ public boolean isMax() { return this == MAX_VALUE; } + public boolean isDummy() { + return this == DUMMY_KEY_DESC; + } + public boolean hasLowerValues() { return lowerValues != null; } @@ -138,6 +144,9 @@ public PartitionKeyValueType getPartitionType() { } public void analyze(int partColNum) throws AnalysisException { + if (isDummy()) { + return; + } if (!isMax()) { if ((upperValues != null && (upperValues.isEmpty() || upperValues.size() > partColNum))) { throw new AnalysisException("Partition values number is more than partition column number: " + toSql()); 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 b3fa6d7b38dd28b..0d3ac83f7253ac3 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 @@ -834,9 +834,6 @@ private Partition dropPartition(long dbId, String partitionName, boolean isForce idToPartition.remove(partition.getId()); nameToPartition.remove(partitionName); - Preconditions.checkState(partitionInfo.getType() == PartitionType.RANGE - || partitionInfo.getType() == PartitionType.LIST); - if (!isForceDrop) { // recycle partition if (partitionInfo.getType() == PartitionType.RANGE) { @@ -936,10 +933,14 @@ public Partition dropPartition(long dbId, String partitionName, boolean isForceD * */ - // get partition by name, not including temp partitions + // Priority is given to querying from the partition. If not found, query from the tempPartition @Override public Partition getPartition(String partitionName) { - return getPartition(partitionName, false); + Partition partition = getPartition(partitionName, false); + if (partition != null) { + return partition; + } + return getPartition(partitionName, true); } // get partition by name @@ -951,7 +952,7 @@ public Partition getPartition(String partitionName, boolean isTempPartition) { } } - // get partition by id, including temp partitions + // Priority is given to querying from the partition. If not found, query from the tempPartition public Partition getPartition(long partitionId) { Partition partition = idToPartition.get(partitionId); if (partition == null) { 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 30837fb685064d7..41143866804ce6c 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 @@ -42,6 +42,7 @@ import org.apache.doris.analysis.HashDistributionDesc; import org.apache.doris.analysis.KeysDesc; import org.apache.doris.analysis.PartitionDesc; +import org.apache.doris.analysis.PartitionKeyDesc; import org.apache.doris.analysis.QueryStmt; import org.apache.doris.analysis.RecoverDbStmt; import org.apache.doris.analysis.RecoverPartitionStmt; @@ -1348,15 +1349,21 @@ public void addPartitionLike(Database db, String tableName, AddPartitionLikeClau + existedName + ". Reason: " + "partition " + existedName + "not exist"); } PartitionInfo partitionInfo = olapTable.getPartitionInfo(); - PartitionDesc partitionDesc = partitionInfo.toPartitionDesc((OlapTable) table); - SinglePartitionDesc oldPartitionDesc = partitionDesc.getSinglePartitionDescByName(existedName); - if (oldPartitionDesc == null) { - throw new DdlException("Failed to ADD PARTITION" + partitionName + " LIKE " - + existedName + ". Reason: " + "partition " + existedName + "desc not exist"); + SinglePartitionDesc newPartitionDesc; + if (partitionInfo instanceof SinglePartitionInfo) { + newPartitionDesc = new SinglePartitionDesc(false, partitionName, + PartitionKeyDesc.DUMMY_KEY_DESC, Maps.newHashMap()); + } else { + PartitionDesc partitionDesc = partitionInfo.toPartitionDesc((OlapTable) table); + SinglePartitionDesc oldPartitionDesc = partitionDesc.getSinglePartitionDescByName(existedName); + if (oldPartitionDesc == null) { + throw new DdlException("Failed to ADD PARTITION" + partitionName + " LIKE " + + existedName + ". Reason: " + "partition " + existedName + "desc not exist"); + } + newPartitionDesc = new SinglePartitionDesc(false, partitionName, + oldPartitionDesc.getPartitionKeyDesc(), oldPartitionDesc.getProperties()); } DistributionDesc distributionDesc = part.getDistributionInfo().toDistributionDesc(); - SinglePartitionDesc newPartitionDesc = new SinglePartitionDesc(false, partitionName, - oldPartitionDesc.getPartitionKeyDesc(), oldPartitionDesc.getProperties()); Map properties = newPartitionDesc.getProperties(); clause = new AddPartitionClause(newPartitionDesc, distributionDesc, properties, addPartitionLikeClause.getIsTempPartition()); @@ -1389,10 +1396,6 @@ public void addPartition(Database db, String tableName, AddPartitionClause addPa olapTable.checkNormalStateForAlter(); // check partition type PartitionInfo partitionInfo = olapTable.getPartitionInfo(); - if (partitionInfo.getType() != PartitionType.RANGE && partitionInfo.getType() != PartitionType.LIST) { - throw new DdlException("Only support adding partition to range and list partitioned table"); - } - // check partition name if (olapTable.checkPartitionNameExist(partitionName)) { if (singlePartitionDesc.isSetIfNotExists()) { @@ -1605,9 +1608,6 @@ public void addPartition(Database db, String tableName, AddPartitionClause addPa // check partition type PartitionInfo partitionInfo = olapTable.getPartitionInfo(); - if (partitionInfo.getType() != PartitionType.RANGE && partitionInfo.getType() != PartitionType.LIST) { - throw new DdlException("Only support adding partition to range and list partitioned table"); - } // update partition info partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, partitionId, isTempPartition); @@ -1630,6 +1630,11 @@ public void addPartition(Database db, String tableName, AddPartitionClause addPa RangePartitionItem.DUMMY_ITEM, partitionInfo.getItem(partitionId), dataProperty, partitionInfo.getReplicaAllocation(partitionId), partitionInfo.getIsInMemory(partitionId), isTempPartition, partitionInfo.getIsMutable(partitionId)); + } else { + info = new PartitionPersistInfo(db.getId(), olapTable.getId(), partition, + RangePartitionItem.DUMMY_ITEM, ListPartitionItem.DUMMY_ITEM, dataProperty, + partitionInfo.getReplicaAllocation(partitionId), partitionInfo.getIsInMemory(partitionId), + isTempPartition, partitionInfo.getIsMutable(partitionId)); } Env.getCurrentEnv().getEditLog().logAddPartition(info); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java index b7306fd44eef757..a385f7164c028a5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java @@ -84,6 +84,10 @@ public List getPartitions() { return partitions; } + public List getHints() { + return hints; + } + public boolean isPartialUpdate() { return isPartialUpdate; } 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 96f0d74f2f8d05c..abb85ad0e1adb39 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 @@ -366,6 +366,7 @@ public LogicalPlan visitStatementDefault(StatementDefaultContext ctx) { @Override public LogicalPlan visitInsertIntoQuery(InsertIntoQueryContext ctx) { + boolean isOverwrite = ctx.INTO() == null; List tableName = visitMultipartIdentifier(ctx.tableName); String labelName = ctx.labelName == null ? null : ctx.labelName.getText(); List colNames = ctx.cols == null ? ImmutableList.of() : visitIdentifierList(ctx.cols); @@ -379,7 +380,7 @@ public LogicalPlan visitInsertIntoQuery(InsertIntoQueryContext ctx) { if (ctx.explain() != null) { return withExplain(sink, ctx.explain()); } - return new InsertIntoTableCommand(sink, Optional.ofNullable(labelName)); + return new InsertIntoTableCommand(sink, Optional.ofNullable(labelName), isOverwrite); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteCommand.java index dba753264ae218c..a124a9fb5aad788 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteCommand.java @@ -70,7 +70,7 @@ public DeleteCommand(List nameParts, String tableAlias, List par @Override public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { - new InsertIntoTableCommand(completeQueryPlan(ctx, logicalQuery), Optional.empty()).run(ctx, executor); + new InsertIntoTableCommand(completeQueryPlan(ctx, logicalQuery), Optional.empty(), false).run(ctx, executor); } private void checkTable(ConnectContext ctx) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java index 3155eab2e7c80ef..feda9f9b0a342a3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java @@ -17,11 +17,23 @@ package org.apache.doris.nereids.trees.plans.commands; +import org.apache.doris.analysis.AddPartitionLikeClause; +import org.apache.doris.analysis.AlterClause; +import org.apache.doris.analysis.AlterTableStmt; import org.apache.doris.analysis.Analyzer; +import org.apache.doris.analysis.DropPartitionClause; +import org.apache.doris.analysis.PartitionNames; +import org.apache.doris.analysis.ReplacePartitionClause; +import org.apache.doris.analysis.TableName; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.OlapTable; import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.UserException; import org.apache.doris.common.util.ProfileManager.ProfileType; +import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.analyzer.UnboundOlapTableSink; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.trees.TreeNode; @@ -34,26 +46,34 @@ import org.apache.doris.nereids.txn.Transaction; import org.apache.doris.planner.OlapTableSink; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.DdlExecutor; import org.apache.doris.qe.QueryState.MysqlStateType; import org.apache.doris.qe.StmtExecutor; import org.apache.doris.transaction.TransactionState; import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import org.apache.commons.collections.CollectionUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.UUID; /** * insert into select command implementation * insert into select command support the grammer: explain? insert into table columns? partitions? hints? query * InsertIntoTableCommand is a command to represent insert the answer of a query into a table. * class structure's: - * InsertIntoTableCommand(Query()) - * ExplainCommand(Query()) + * InsertIntoTableCommand(Query()) + * ExplainCommand(Query()) */ public class InsertIntoTableCommand extends Command implements ForwardWithSync, Explainable { @@ -62,16 +82,18 @@ public class InsertIntoTableCommand extends Command implements ForwardWithSync, private final LogicalPlan logicalQuery; private final Optional labelName; private NereidsPlanner planner; + private boolean isOverwrite; private boolean isTxnBegin = false; /** * constructor */ - public InsertIntoTableCommand(LogicalPlan logicalQuery, Optional labelName) { + public InsertIntoTableCommand(LogicalPlan logicalQuery, Optional labelName, boolean isOverwrite) { super(PlanType.INSERT_INTO_TABLE_COMMAND); this.logicalQuery = Objects.requireNonNull(logicalQuery, "logicalQuery cannot be null in InsertIntoTableCommand"); this.labelName = labelName; + this.isOverwrite = isOverwrite; } public NereidsPlanner getPlanner() { @@ -88,6 +110,7 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { } throw new AnalysisException("Nereids DML is disabled, will try to fall back to the original planner"); } + if (ctx.isTxnModel()) { // in original planner, if is in txn model, insert into select command and tableRef >= 1 will be refused. // we can just run select a one-row-relation like select 1, 2, 3 @@ -111,6 +134,11 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { Preconditions.checkArgument(plan.isPresent(), "insert into command must contain OlapTableSinkNode"); PhysicalOlapTableSink physicalOlapTableSink = ((PhysicalOlapTableSink) plan.get()); + if (isOverwrite) { + dealOverwrite(ctx, executor, physicalOlapTableSink); + return; + } + OlapTableSink sink = ((OlapTableSink) planner.getFragments().get(0).getSink()); Preconditions.checkArgument(!isTxnBegin, "an insert command cannot create more than one txn"); @@ -151,6 +179,153 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { } } + /** + * when `isOverwrite` is true, use this logic + * + * @param ctx ctx + * @param executor executor + * @param physicalOlapTableSink physicalOlapTableSink + * @throws Exception Exception + */ + public void dealOverwrite(ConnectContext ctx, StmtExecutor executor, PhysicalOlapTableSink physicalOlapTableSink) + throws Exception { + OlapTable targetTable = physicalOlapTableSink.getTargetTable(); + TableName tableName = new TableName(InternalCatalog.INTERNAL_CATALOG_NAME, targetTable.getQualifiedDbName(), + targetTable.getName()); + List partitionNames = ((UnboundOlapTableSink) logicalQuery).getPartitions(); + if (CollectionUtils.isEmpty(partitionNames)) { + partitionNames = Lists.newArrayList(targetTable.getPartitionNames()); + } + List tempPartitionNames = addTempPartition(ctx, tableName, partitionNames); + boolean insertRes = insertInto(ctx, executor, tempPartitionNames, tableName); + if (!insertRes) { + return; + } + replacePartition(ctx, tableName, partitionNames, tempPartitionNames); + } + + /** + * replacing partitionNames with tempPartitionNames + * + * @param ctx ctx + * @param tableName tableName + * @param partitionNames partitionNames + * @param tempPartitionNames tempPartitionNames + * @throws UserException UserException + */ + private void replacePartition(ConnectContext ctx, TableName tableName, List partitionNames, + List tempPartitionNames) + throws UserException { + // overwrite old partition with tmp partition + try { + List ops = new ArrayList<>(); + Map properties = new HashMap<>(); + properties.put("use_temp_partition_name", "false"); + ops.add(new ReplacePartitionClause(new PartitionNames(false, partitionNames), + new PartitionNames(true, tempPartitionNames), properties)); + AlterTableStmt alterTableStmt = new AlterTableStmt(tableName, ops); + Env.getCurrentEnv().alterTable(alterTableStmt); + } catch (Exception e) { + LOG.warn("IOT overwrite table partitions error", e); + handleIotPartitionRollback(ctx, tableName, tempPartitionNames); + throw e; + } + } + + /** + * insert into select + * + * @param ctx ctx + * @param executor executor + * @param tempPartitionNames tempPartitionNames + * @param tableName tableName + * @throws Exception Exception + */ + private boolean insertInto(ConnectContext ctx, StmtExecutor executor, List tempPartitionNames, + TableName tableName) { + try { + UnboundOlapTableSink sink = (UnboundOlapTableSink) logicalQuery; + UnboundOlapTableSink copySink = new UnboundOlapTableSink<>( + sink.getNameParts(), + sink.getColNames(), + sink.getHints(), + tempPartitionNames, + (LogicalPlan) (sink.child(0))); + new InsertIntoTableCommand(copySink, labelName, false).run(ctx, executor); + if (ctx.getState().getStateType() == MysqlStateType.ERR) { + String errMsg = Strings.emptyToNull(ctx.getState().getErrorMessage()); + LOG.warn("InsertInto state error:{}", errMsg); + handleIotPartitionRollback(ctx, tableName, tempPartitionNames); + return false; + } + return true; + } catch (Exception e) { + LOG.warn("InsertInto error", e); + handleIotPartitionRollback(ctx, tableName, tempPartitionNames); + return false; + } + } + + /** + * add some tempPartitions + * + * @param ctx ctx + * @param tableName tableName + * @param partitionNames partitionNames + * @return tempPartitionNames + * @throws Exception Exception + */ + private List addTempPartition(ConnectContext ctx, TableName tableName, List partitionNames) + throws Exception { + List tempPartitionNames = new ArrayList<>(); + try { + // create tmp partitions with uuid + for (String partitionName : partitionNames) { + UUID uuid = UUID.randomUUID(); + // to comply with naming rules + String tempPartName = "tmp_partition_" + uuid.toString().replace('-', '_'); + List ops = new ArrayList<>(); + ops.add(new AddPartitionLikeClause(tempPartName, partitionName, true)); + + AlterTableStmt alterTableStmt = new AlterTableStmt(tableName, ops); + Analyzer tempAnalyzer = new Analyzer(Env.getCurrentEnv(), ctx); + alterTableStmt.analyze(tempAnalyzer); + DdlExecutor.execute(ctx.getEnv(), alterTableStmt); + // only when execution succeeded, put the temp partition name into list + tempPartitionNames.add(tempPartName); + } + return tempPartitionNames; + } catch (Exception e) { + LOG.warn("IOT create tmp table partitions error", e); + handleIotPartitionRollback(ctx, tableName, tempPartitionNames); + throw e; + } + } + + /** + * delete temp partitions + * + * @param ctx ctx + * @param targetTableName targetTableName + * @param tempPartitionNames tempPartitionNames + */ + private void handleIotPartitionRollback(ConnectContext ctx, TableName targetTableName, + List tempPartitionNames) { + try { + for (String partitionName : tempPartitionNames) { + List ops = new ArrayList<>(); + ops.add(new DropPartitionClause(true, partitionName, true, true)); + AlterTableStmt dropTablePartitionStmt = new AlterTableStmt(targetTableName, ops); + Analyzer tempAnalyzer = new Analyzer(Env.getCurrentEnv(), ctx); + dropTablePartitionStmt.analyze(tempAnalyzer); + DdlExecutor.execute(ctx.getEnv(), dropTablePartitionStmt); + } + } catch (Exception ex) { + LOG.warn("IOT drop partitions error", ex); + ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, "Unexpected exception: " + ex.getMessage()); + } + } + @Override public Plan getExplainPlan(ConnectContext ctx) { return this.logicalQuery; 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 6236ba019b5f9b4..0a0b3fd9a068286 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 @@ -88,7 +88,7 @@ public UpdateCommand(List nameParts, @Nullable String tableAlias, List Date: Thu, 7 Sep 2023 09:55:29 +0800 Subject: [PATCH 03/14] [improvement](bitmap) support version for ser/deser of bitmap (#23959) --- be/src/common/config.cpp | 2 + be/src/common/config.h | 3 + be/src/util/bitmap_value.h | 89 ++++++--- be/test/util/bitmap_value_test.cpp | 8 +- be/test/vec/function/function_bitmap_test.cpp | 170 ++++++++++++------ 5 files changed, 190 insertions(+), 82 deletions(-) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 2719046a03a9816..4c3099473306699 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1082,6 +1082,8 @@ DEFINE_Int32(fe_expire_duration_seconds, "60"); DEFINE_Int32(grace_shutdown_wait_seconds, "120"); +DEFINE_Int16(bitmap_serialize_version, "1"); + #ifdef BE_TEST // test s3 DEFINE_String(test_s3_resource, "resource"); diff --git a/be/src/common/config.h b/be/src/common/config.h index 95c4d219e0a4449..78ee4f9fefe40a3 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1150,6 +1150,9 @@ DECLARE_Int32(fe_expire_duration_seconds); // During this period, FE will not send any queries to BE and waiting for all running queries to stop. DECLARE_Int32(grace_shutdown_wait_seconds); +// BitmapValue serialize version. +DECLARE_Int16(bitmap_serialize_version); + #ifdef BE_TEST // test s3 DECLARE_String(test_s3_resource); diff --git a/be/src/util/bitmap_value.h b/be/src/util/bitmap_value.h index e67e665155071ab..041afa7e66dcb55 100644 --- a/be/src/util/bitmap_value.h +++ b/be/src/util/bitmap_value.h @@ -76,10 +76,14 @@ struct BitmapTypeCode { // // added in 0.12 BITMAP64 = 4, - SET = 5 + SET = 5, // V1 + SET_V2 = 10, + BITMAP32_V2 = 12, + BITMAP64_V2 = 13, + TYPE_MAX }; Status static inline validate(int bitmap_type) { - if (UNLIKELY(bitmap_type < type::EMPTY || bitmap_type > type::SET)) { + if (UNLIKELY(bitmap_type < type::EMPTY || bitmap_type >= type::TYPE_MAX)) { std::string err_msg = fmt::format("BitmapTypeCode invalid, should between: {} and {} actrual is {}", BitmapTypeCode::EMPTY, BitmapTypeCode::BITMAP64, bitmap_type); @@ -699,29 +703,35 @@ class Roaring64Map { * write a bitmap to a char buffer. * Returns how many bytes were written which should be getSizeInBytes(). */ - size_t write(char* buf) const { + size_t write(char* buf, int serialize_version) const { + bool is_v1 = serialize_version == 1; + BitmapTypeCode::type type_bitmap32 = + is_v1 ? BitmapTypeCode::type::BITMAP32 : BitmapTypeCode::type::BITMAP32_V2; + BitmapTypeCode::type type_bitmap64 = + is_v1 ? BitmapTypeCode::type::BITMAP64 : BitmapTypeCode::type::BITMAP64_V2; + if (is32BitsEnough()) { - *(buf++) = BitmapTypeCode::type::BITMAP32; + *(buf++) = type_bitmap32; auto it = roarings.find(0); if (it == roarings.end()) { // empty bitmap roaring::Roaring r; - return r.write(buf) + 1; + return r.write(buf, is_v1) + 1; } - return it->second.write(buf) + 1; + return it->second.write(buf, is_v1) + 1; } const char* orig = buf; // put type code - *(buf++) = BitmapTypeCode::type::BITMAP64; + *(buf++) = type_bitmap64; // push map size buf = (char*)encode_varint64((uint8_t*)buf, roarings.size()); std::for_each(roarings.cbegin(), roarings.cend(), - [&buf](const std::pair& map_entry) { + [&buf, is_v1](const std::pair& map_entry) { // push map key encode_fixed32_le((uint8_t*)buf, map_entry.first); buf += sizeof(uint32_t); // push map value Roaring - buf += map_entry.second.write(buf); + buf += map_entry.second.write(buf, is_v1); }); return buf - orig; } @@ -735,13 +745,16 @@ class Roaring64Map { static Roaring64Map read(const char* buf) { Roaring64Map result; - if (*buf == BitmapTypeCode::BITMAP32) { - roaring::Roaring read = roaring::Roaring::read(buf + 1); + bool is_v1 = BitmapTypeCode::BITMAP32 == *buf || BitmapTypeCode::BITMAP64 == *buf; + bool is_bitmap32 = BitmapTypeCode::BITMAP32 == *buf || BitmapTypeCode::BITMAP32_V2 == *buf; + bool is_bitmap64 = BitmapTypeCode::BITMAP64 == *buf || BitmapTypeCode::BITMAP64_V2 == *buf; + if (is_bitmap32) { + roaring::Roaring read = roaring::Roaring::read(buf + 1, is_v1); result.emplaceOrInsert(0, std::move(read)); return result; } - DCHECK_EQ(BitmapTypeCode::BITMAP64, *buf); + DCHECK(is_bitmap64); buf++; // get map size (varint64 took 1~10 bytes) @@ -755,9 +768,9 @@ class Roaring64Map { uint32_t key = decode_fixed32_le(reinterpret_cast(buf)); buf += sizeof(uint32_t); // read map value Roaring - roaring::Roaring read_var = roaring::Roaring::read(buf); + roaring::Roaring read_var = roaring::Roaring::read(buf, is_v1); // forward buffer past the last Roaring Bitmap - buf += read_var.getSizeInBytes(); + buf += read_var.getSizeInBytes(is_v1); result.emplaceOrInsert(key, std::move(read_var)); } return result; @@ -766,14 +779,15 @@ class Roaring64Map { /** * How many bytes are required to serialize this bitmap */ - size_t getSizeInBytes() const { + size_t getSizeInBytes(int serialize_version) const { + bool is_v1 = serialize_version == 1; if (is32BitsEnough()) { auto it = roarings.find(0); if (it == roarings.end()) { // empty bitmap roaring::Roaring r; - return r.getSizeInBytes() + 1; + return r.getSizeInBytes(is_v1) + 1; } - return it->second.getSizeInBytes() + 1; + return it->second.getSizeInBytes(is_v1) + 1; } // start with type code, map size and size of keys for each map entry size_t init = 1 + varint_length(roarings.size()) + roarings.size() * sizeof(uint32_t); @@ -781,7 +795,7 @@ class Roaring64Map { roarings.cbegin(), roarings.cend(), init, [=](size_t previous, const std::pair& map_entry) { // add in bytes used by each Roaring - return previous + map_entry.second.getSizeInBytes(); + return previous + map_entry.second.getSizeInBytes(is_v1); }); } @@ -1314,10 +1328,11 @@ class BitmapValue { case SET: return BitmapTypeCode::SET; case BITMAP: + bool is_v1 = (config::bitmap_serialize_version == 1); if (_bitmap->is32BitsEnough()) { - return BitmapTypeCode::BITMAP32; + return is_v1 ? BitmapTypeCode::type::BITMAP32 : BitmapTypeCode::type::BITMAP32_V2; } else { - return BitmapTypeCode::BITMAP64; + return is_v1 ? BitmapTypeCode::type::BITMAP64 : BitmapTypeCode::type::BITMAP64_V2; } } } @@ -2167,7 +2182,7 @@ class BitmapValue { case BITMAP: _bitmap->runOptimize(); _bitmap->shrinkToFit(); - res = _bitmap->getSizeInBytes(); + res = _bitmap->getSizeInBytes(config::bitmap_serialize_version); break; case SET: /// 1 byte for type, 1 byte for count @@ -2205,7 +2220,7 @@ class BitmapValue { } break; case BITMAP: - _bitmap->write(dst); + _bitmap->write(dst, config::bitmap_serialize_version); break; } } @@ -2235,6 +2250,8 @@ class BitmapValue { break; case BitmapTypeCode::BITMAP32: case BitmapTypeCode::BITMAP64: + case BitmapTypeCode::BITMAP32_V2: + case BitmapTypeCode::BITMAP64_V2: _type = BITMAP; _prepare_bitmap_for_write(); *_bitmap = detail::Roaring64Map::read(src); @@ -2260,6 +2277,34 @@ class BitmapValue { } break; } + case BitmapTypeCode::SET_V2: { + uint32_t size = 0; + memcpy(&size, src + 1, sizeof(uint32_t)); + src += sizeof(uint32_t) + 1; + + if (!config::enable_set_in_bitmap_value || size > SET_TYPE_THRESHOLD) { + _type = BITMAP; + _prepare_bitmap_for_write(); + + for (int i = 0; i < size; ++i) { + uint64_t key {}; + memcpy(&key, src, sizeof(uint64_t)); + _bitmap->add(key); + src += sizeof(uint64_t); + } + } else { + _type = SET; + _set.reserve(size); + + for (int i = 0; i < size; ++i) { + uint64_t key {}; + memcpy(&key, src, sizeof(uint64_t)); + _set.insert(key); + src += sizeof(uint64_t); + } + } + break; + } default: LOG(ERROR) << "BitmapTypeCode invalid, should between: " << BitmapTypeCode::EMPTY << " and " << BitmapTypeCode::BITMAP64 << " actual is " diff --git a/be/test/util/bitmap_value_test.cpp b/be/test/util/bitmap_value_test.cpp index dc9908a5823b385..416137bb295811a 100644 --- a/be/test/util/bitmap_value_test.cpp +++ b/be/test/util/bitmap_value_test.cpp @@ -266,9 +266,9 @@ TEST(BitmapValueTest, Roaring64Map) { } EXPECT_TRUE(r1.contains((uint64_t)14000000000000000500ull)); EXPECT_EQ(1800, r1.cardinality()); - size_t size_before = r1.getSizeInBytes(); + size_t size_before = r1.getSizeInBytes(1); r1.runOptimize(); - size_t size_after = r1.getSizeInBytes(); + size_t size_after = r1.getSizeInBytes(1); EXPECT_LT(size_after, size_before); Roaring64Map r2 = Roaring64Map::bitmapOf(5, 1ull, 2ull, 234294967296ull, 195839473298ull, @@ -311,9 +311,9 @@ TEST(BitmapValueTest, Roaring64Map) { EXPECT_EQ(1, i1_2.cardinality()); // we can write a bitmap to a pointer and recover it later - uint32_t expectedsize = r1.getSizeInBytes(); + uint32_t expectedsize = r1.getSizeInBytes(1); char* serializedbytes = new char[expectedsize]; - r1.write(serializedbytes); + r1.write(serializedbytes, 1); Roaring64Map t = Roaring64Map::read(serializedbytes); EXPECT_TRUE(r1 == t); delete[] serializedbytes; diff --git a/be/test/vec/function/function_bitmap_test.cpp b/be/test/vec/function/function_bitmap_test.cpp index 34d74a549d0cd8e..312a10ca4ec6dad 100644 --- a/be/test/vec/function/function_bitmap_test.cpp +++ b/be/test/vec/function/function_bitmap_test.cpp @@ -93,6 +93,11 @@ TEST(function_bitmap_test, function_bitmap_to_base64) { config::Register::_s_field_map->insert( std::make_pair(std::string("enable_set_in_bitmap_value"), field)); + config::Register::Field field_ser_ver("int16_t", "bitmap_serialize_version", + &config::bitmap_serialize_version, "1", false); + config::Register::_s_field_map->insert( + std::make_pair(std::string("bitmap_serialize_version"), field_ser_ver)); + std::string func_name = "bitmap_to_base64"; InputTypeSet input_types = {TypeIndex::BitMap}; @@ -123,19 +128,21 @@ TEST(function_bitmap_test, function_bitmap_to_base64) { EXPECT_EQ(bitmap64_2.get_type_code(), BitmapTypeCode::BITMAP64); EXPECT_EQ(bitmap64_3.get_type_code(), BitmapTypeCode::BITMAP64); - DataSet data_set = { - {{&bitmap32_1}, std::string("AQEAAAA=")}, - {{&bitmap32_2}, std::string("AjowAAACAAAAAAAAAJgAAAAYAAAAGgAAAAEAf5Y=")}, - {{&bitmap32_3}, std::string("AjswAAABAAAgAAEAAAAgAA==")}, - {{&bitmap64_1}, std::string("AwAAAAABAAAA")}, - {{&bitmap64_2}, - std::string("BAIAAAAAOjAAAAEAAAAAAAAAEAAAAAEAAQAAADowAAABAAAAAAAAABAAAAAAAA==")}, - {{&bitmap64_3}, - std::string("BAIAAAAAOzAAAAEAAB8AAQAAAB8AAQAAADowAAABAAAAAAAAABAAAAAAAA==")}, - {{&empty_bitmap}, std::string("AA==")}, - {{Null()}, Null()}}; - - check_function(func_name, input_types, data_set); + { + DataSet data_set = { + {{&bitmap32_1}, std::string("AQEAAAA=")}, + {{&bitmap32_2}, std::string("AjowAAACAAAAAAAAAJgAAAAYAAAAGgAAAAEAf5Y=")}, + {{&bitmap32_3}, std::string("AjswAAABAAAgAAEAAAAgAA==")}, + {{&bitmap64_1}, std::string("AwAAAAABAAAA")}, + {{&bitmap64_2}, + std::string("BAIAAAAAOjAAAAEAAAAAAAAAEAAAAAEAAQAAADowAAABAAAAAAAAABAAAAAAAA==")}, + {{&bitmap64_3}, + std::string("BAIAAAAAOzAAAAEAAB8AAQAAAB8AAQAAADowAAABAAAAAAAAABAAAAAAAA==")}, + {{&empty_bitmap}, std::string("AA==")}, + {{Null()}, Null()}}; + + check_function(func_name, input_types, data_set); + } EXPECT_TRUE(config::set_config("enable_set_in_bitmap_value", "true", false, true).ok()); bitmap32_1 = BitmapValue(1); // single @@ -154,18 +161,46 @@ TEST(function_bitmap_test, function_bitmap_to_base64) { EXPECT_EQ(bitmap64_2.get_type_code(), BitmapTypeCode::SET); EXPECT_EQ(bitmap64_3.get_type_code(), BitmapTypeCode::BITMAP64); - DataSet data_set2 = { - {{&bitmap32_1}, std::string("AQEAAAA=")}, - {{&bitmap32_2}, std::string("BQIBAAAAAAAAAH+WmAAAAAAA")}, - {{&bitmap32_3}, std::string("AjswAAABAAAgAAEAAAAgAA==")}, - {{&bitmap64_1}, std::string("AwAAAAABAAAA")}, - {{&bitmap64_2}, std::string("BQIAAAAAAQAAAAEAAAAAAAAA")}, - {{&bitmap64_3}, - std::string("BAIAAAAAOzAAAAEAAB8AAQAAAB8AAQAAADowAAABAAAAAAAAABAAAAAAAA==")}, - {{&empty_bitmap}, std::string("AA==")}, - {{Null()}, Null()}}; - - check_function(func_name, input_types, data_set2); + { + DataSet data_set = { + {{&bitmap32_1}, std::string("AQEAAAA=")}, + {{&bitmap32_2}, std::string("BQIBAAAAAAAAAH+WmAAAAAAA")}, + {{&bitmap32_3}, std::string("AjswAAABAAAgAAEAAAAgAA==")}, + {{&bitmap64_1}, std::string("AwAAAAABAAAA")}, + {{&bitmap64_2}, std::string("BQIAAAAAAQAAAAEAAAAAAAAA")}, + {{&bitmap64_3}, + std::string("BAIAAAAAOzAAAAEAAB8AAQAAAB8AAQAAADowAAABAAAAAAAAABAAAAAAAA==")}, + {{&empty_bitmap}, std::string("AA==")}, + {{Null()}, Null()}}; + + check_function(func_name, input_types, data_set); + } + + { + std::string base64("BQQAAAAAAAAAAAEAAAAAAAAAAgAAAAAAAAADAAAAAAAAAA=="); + BitmapValue bitmap; + bitmap.add(0); + bitmap.add(1); + bitmap.add(2); + bitmap.add(3); + DataSet data_set = {{{&bitmap}, base64}}; + check_function(func_name, input_types, data_set); + } + + // test bitmap serialize version2 + EXPECT_TRUE(config::set_config("bitmap_serialize_version", "2", false, true).ok()); + bitmap32_3 = BitmapValue(bits32); // bitmap32 + bitmap64_3 = BitmapValue(bits64); // bitmap64 + EXPECT_EQ(bitmap32_3.get_type_code(), BitmapTypeCode::BITMAP32_V2); + EXPECT_EQ(bitmap64_3.get_type_code(), BitmapTypeCode::BITMAP64_V2); + + { + DataSet data_set = { + {{&bitmap32_3}, std::string("DAI7MAAAAQAAIAABAAAAIAA=")}, + {{&bitmap64_3}, std::string("DQIAAAAAAjswAAABAAAfAAEAAAAfAAEAAAABAQAAAAAAAAA=")}}; + + check_function(func_name, input_types, data_set); + } } TEST(function_bitmap_test, function_bitmap_from_base64) { @@ -174,6 +209,11 @@ TEST(function_bitmap_test, function_bitmap_from_base64) { config::Register::_s_field_map->insert( std::make_pair(std::string("enable_set_in_bitmap_value"), field)); + config::Register::Field field_ser_ver("int16_t", "bitmap_serialize_version", + &config::bitmap_serialize_version, "1", false); + config::Register::_s_field_map->insert( + std::make_pair(std::string("bitmap_serialize_version"), field_ser_ver)); + std::string func_name = "bitmap_from_base64"; InputTypeSet input_types = {TypeIndex::String}; @@ -205,47 +245,65 @@ TEST(function_bitmap_test, function_bitmap_from_base64) { BitmapValue bitmap64_3(bits64); // bitmap BitmapValue empty_bitmap; - DataSet data_set = {{{bitmap32_base64_1}, bitmap32_1}, {{bitmap32_base64_2}, bitmap32_2}, - {{bitmap32_base64_3}, bitmap32_3}, {{bitmap64_base64_1}, bitmap64_1}, - {{bitmap64_base64_2}, bitmap64_2}, {{bitmap64_base64_3}, bitmap64_3}, - {{base64_empty}, empty_bitmap}, {{Null()}, Null()}}; + { + DataSet data_set = {{{bitmap32_base64_1}, bitmap32_1}, {{bitmap32_base64_2}, bitmap32_2}, + {{bitmap32_base64_3}, bitmap32_3}, {{bitmap64_base64_1}, bitmap64_1}, + {{bitmap64_base64_2}, bitmap64_2}, {{bitmap64_base64_3}, bitmap64_3}, + {{base64_empty}, empty_bitmap}, {{Null()}, Null()}}; - check_function(func_name, input_types, data_set); + check_function(func_name, input_types, data_set); + } EXPECT_TRUE(config::set_config("enable_set_in_bitmap_value", "true", false, true).ok()); bitmap32_base64_1 = ("AQEAAAA="); - bitmap32_base64_2 = ("BQIBAAAAAAAAAH"); + bitmap32_base64_2 = ("BQIBAAAAAAAAAH+WmAAAAAAA"); bitmap32_base64_3 = ("AjswAAABAAAgAAEAAAAgAA=="); bitmap64_base64_1 = ("AwAAAAABAAAA"); bitmap64_base64_2 = ("BQIAAAAAAQAAAAEAAAAAAAAA"); bitmap64_base64_3 = ("BAIAAAAAOzAAAAEAAB8AAQAAAB8AAQAAADowAAABAAAAAAAAABAAAAAAAA=="); - check_function(func_name, input_types, data_set); - - /* sr - mysql [(none)]>select bitmap_to_base64(bitmap_from_string("0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32")); - +----------------------------------------------------------------------------------------------------------------------------------+ - | bitmap_to_base64(bitmap_from_string('0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32')) | - +----------------------------------------------------------------------------------------------------------------------------------+ - | AjowAAABAAAAAAAgABAAAAAAAAEAAgADAAQABQAGAAcACAAJAAoACwAMAA0ADgAPABAAEQASABMAFAAVABYAFwAYABkAGgAbABwAHQAeAB8AIAA= | - +----------------------------------------------------------------------------------------------------------------------------------+ - - mysql [(none)]>select bitmap_to_base64(bitmap_from_string("0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,4294967296")); - +--------------------------------------------------------------------------------------------------------------------------------------------------+ - | bitmap_to_base64(bitmap_from_string('0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,4294967296')) | - +--------------------------------------------------------------------------------------------------------------------------------------------------+ - | BAIAAAAAOjAAAAEAAAAAAB8AEAAAAAAAAQACAAMABAAFAAYABwAIAAkACgALAAwADQAOAA8AEAARABIAEwAUABUAFgAXABgAGQAaABsAHAAdAB4AHwABAAAAOjAAAAEAAAAAAAAAEAAAAAAA | - +--------------------------------------------------------------------------------------------------------------------------------------------------+ - */ - bitmap32_base64_3 = - ("AjowAAABAAAAAAAgABAAAAAAAAEAAgADAAQABQAGAAcACAAJAAoACwAMAA0ADgAPABAAEQASABMAFAAVABYAF" - "wAYABkAGgAbABwAHQAeAB8AIAA="); - bitmap64_base64_3 = - ("BAIAAAAAOjAAAAEAAAAAAB8AEAAAAAAAAQACAAMABAAFAAYABwAIAAkACgALAAwADQAOAA8AEAARABIAEwAUA" - "BUAFgAXABgAGQAaABsAHAAdAB4AHwABAAAAOjAAAAEAAAAAAAAAEAAAAAAA"); - data_set = {{{bitmap32_base64_3}, bitmap32_3}, {{bitmap64_base64_3}, bitmap64_3}}; - check_function(func_name, input_types, data_set); + { + DataSet data_set = {{{bitmap32_base64_1}, bitmap32_1}, {{bitmap32_base64_2}, bitmap32_2}, + {{bitmap32_base64_3}, bitmap32_3}, {{bitmap64_base64_1}, bitmap64_1}, + {{bitmap64_base64_2}, bitmap64_2}, {{bitmap64_base64_3}, bitmap64_3}, + {{base64_empty}, empty_bitmap}, {{Null()}, Null()}}; + + check_function(func_name, input_types, data_set); + } + + { + std::string base64("CgIAAAAAAAAAAAAAAAEAAAAAAAAA"); + BitmapValue bitmap; + bitmap.add(0); + bitmap.add(1); + DataSet data_set = {{{base64}, bitmap}}; + check_function(func_name, input_types, data_set); + } + { + EXPECT_TRUE(config::set_config("bitmap_serialize_version", "1", false, true).ok()); + + std::string base64_32_v1( + "AjowAAABAAAAAAAgABAAAAAAAAEAAgADAAQABQAGAAcACAAJAAoACwAMAA0ADgAPABAAEQASABMAFAAVAB" + "YAFwAYABkAGgAbABwAHQAeAB8AIAA="); + std::string base64_64_v1( + "BAIAAAAAOjAAAAEAAAAAAB8AEAAAAAAAAQACAAMABAAFAAYABwAIAAkACgALAAwADQAOAA8AEAARABIAEw" + "AUABUAFgAXABgAGQAaABsAHAAdAB4AHwABAAAAOjAAAAEAAAAAAAAAEAAAAAAA"); + DataSet data_set = {{{base64_32_v1}, bitmap32_3}, {{base64_64_v1}, bitmap64_3}}; + check_function(func_name, input_types, data_set); + } + { + EXPECT_TRUE(config::set_config("bitmap_serialize_version", "2", false, true).ok()); + + std::string base64_32_v2( + "DAI6MAAAAQAAAAAAIAAQAAAAAAABAAIAAwAEAAUABgAHAAgACQAKAAsADAANAA4ADwAQABEAEgATABQAFQ" + "AWABcAGAAZABoAGwAcAB0AHgAfACAA"); + std::string base64_64_v2( + "DQIAAAAAAjowAAABAAAAAAAfABAAAAAAAAEAAgADAAQABQAGAAcACAAJAAoACwAMAA0ADgAPABAAEQASAB" + "MAFAAVABYAFwAYABkAGgAbABwAHQAeAB8AAQAAAAEBAAAAAAAAAA=="); + DataSet data_set = {{{base64_32_v2}, bitmap32_3}, {{base64_64_v2}, bitmap64_3}}; + check_function(func_name, input_types, data_set); + } } TEST(function_bitmap_test, function_bitmap_and_count) { From 537369f4e22b12c530d5b51188df397d0e43318e Mon Sep 17 00:00:00 2001 From: Jack Drogon Date: Thu, 7 Sep 2023 10:10:51 +0800 Subject: [PATCH 04/14] [Fix](http) Fix curl return HTTP_ERROR && Add not_found HttpClientTest, fix (#23984) Signed-off-by: Jack Drogon --- be/src/http/http_client.cpp | 6 +++--- be/src/service/backend_service.cpp | 2 ++ be/test/http/http_client_test.cpp | 34 ++++++++++++++++++++++++++---- 3 files changed, 35 insertions(+), 7 deletions(-) diff --git a/be/src/http/http_client.cpp b/be/src/http/http_client.cpp index 9c3298632c2b94c..d7a6c9c96659255 100644 --- a/be/src/http/http_client.cpp +++ b/be/src/http/http_client.cpp @@ -172,7 +172,7 @@ Status HttpClient::execute(const std::functionhead()); return client->get_content_length(&segment_file_size); }; + status = HttpClient::execute_with_retry(max_retry, 1, get_segment_file_size_cb); if (!status.ok()) { LOG(WARNING) << "failed to get segment file size from " << get_segment_file_size_url @@ -571,6 +572,7 @@ void BackendService::ingest_binlog(TIngestBinlogResult& result, status.to_thrift(&tstatus); return; } + segment_file_sizes.push_back(segment_file_size); segment_file_urls.push_back(std::move(get_segment_file_size_url)); } diff --git a/be/test/http/http_client_test.cpp b/be/test/http/http_client_test.cpp index d802ee7c569e8b1..ae56bd9712fca9c 100644 --- a/be/test/http/http_client_test.cpp +++ b/be/test/http/http_client_test.cpp @@ -73,12 +73,21 @@ class HttpClientTestSimplePostHandler : public HttpHandler { } }; -static HttpClientTestSimpleGetHandler s_simple_get_handler = HttpClientTestSimpleGetHandler(); -static HttpClientTestSimplePostHandler s_simple_post_handler = HttpClientTestSimplePostHandler(); +class HttpNotFoundHandler : public HttpHandler { +public: + void handle(HttpRequest* req) override { + HttpChannel::send_reply(req, HttpStatus::NOT_FOUND, "file not exist."); + } +}; + static EvHttpServer* s_server = nullptr; static int real_port = 0; static std::string hostname = ""; +static HttpClientTestSimpleGetHandler s_simple_get_handler; +static HttpClientTestSimplePostHandler s_simple_post_handler; +static HttpNotFoundHandler s_not_found_handler; + class HttpClientTest : public testing::Test { public: HttpClientTest() {} @@ -89,6 +98,7 @@ class HttpClientTest : public testing::Test { s_server->register_handler(GET, "/simple_get", &s_simple_get_handler); s_server->register_handler(HEAD, "/simple_get", &s_simple_get_handler); s_server->register_handler(POST, "/simple_post", &s_simple_post_handler); + s_server->register_handler(GET, "/not_found", &s_not_found_handler); s_server->start(); real_port = s_server->get_real_port(); EXPECT_NE(0, real_port); @@ -143,10 +153,9 @@ TEST_F(HttpClientTest, get_failed) { auto st = client.init(hostname + "/simple_get"); EXPECT_TRUE(st.ok()); client.set_method(GET); - client.set_basic_auth("test1", ""); std::string response; st = client.execute(&response); - EXPECT_FALSE(!st.ok()); + EXPECT_FALSE(st.ok()); } TEST_F(HttpClientTest, post_normal) { @@ -177,4 +186,21 @@ TEST_F(HttpClientTest, post_failed) { EXPECT_TRUE(boost::algorithm::contains(st.to_string(), not_found)); } +TEST_F(HttpClientTest, not_found) { + HttpClient client; + std::string url = hostname + "/not_found"; + constexpr uint64_t kMaxTimeoutMs = 1000; + + auto get_cb = [&url](HttpClient* client) { + std::string resp; + RETURN_IF_ERROR(client->init(url)); + client->set_timeout_ms(kMaxTimeoutMs); + return client->execute(&resp); + }; + + auto status = HttpClient::execute_with_retry(3, 1, get_cb); + // libcurl is configured by CURLOPT_FAILONERROR + EXPECT_FALSE(status.ok()); +} + } // namespace doris From 45dfbeb8b313d1a7a290570e31af2ad7b2d2a39d Mon Sep 17 00:00:00 2001 From: xy Date: Thu, 7 Sep 2023 10:21:55 +0800 Subject: [PATCH 05/14] [feature](agg) add the aggregation function 'array_agg' #23474 This function requires one arguments just as ARRAY_AGG(col) and col means the column whose values you want to aggregate. This function Aggregates the values including NULL in a column into an array and returns a value of the ARRAY data type. --- .../aggregate_function_collect.cpp | 58 ++-- .../aggregate_function_collect.h | 317 +++++++++++++++++- .../aggregate-functions/array-agg.md | 84 +++++ .../aggregate-functions/array-agg.md | 84 +++++ .../doris/analysis/FunctionCallExpr.java | 3 +- .../doris/catalog/AggregateFunction.java | 3 +- .../org/apache/doris/catalog/FunctionSet.java | 5 + .../data/query_p0/aggregate/array_agg.out | 67 ++++ .../query_p0/aggregate/array_agg.groovy | 195 +++++++++++ 9 files changed, 778 insertions(+), 38 deletions(-) create mode 100644 docs/en/docs/sql-manual/sql-functions/aggregate-functions/array-agg.md create mode 100644 docs/zh-CN/docs/sql-manual/sql-functions/aggregate-functions/array-agg.md create mode 100644 regression-test/data/query_p0/aggregate/array_agg.out create mode 100644 regression-test/suites/query_p0/aggregate/array_agg.groovy diff --git a/be/src/vec/aggregate_functions/aggregate_function_collect.cpp b/be/src/vec/aggregate_functions/aggregate_function_collect.cpp index d6aab144114d338..2831f39aa30eba1 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_collect.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_collect.cpp @@ -27,21 +27,29 @@ namespace doris::vectorized { -template +template AggregateFunctionPtr do_create_agg_function_collect(bool distinct, const DataTypes& argument_types, const bool result_is_nullable) { + if (argument_types[0]->is_nullable()) { + if constexpr (ShowNull::value) { + return creator_without_type::create_ignore_nullable, std::false_type, std::true_type>>( + argument_types, result_is_nullable); + } + } + if (distinct) { - return creator_without_type::create< - AggregateFunctionCollect, HasLimit>>( + return creator_without_type::create, HasLimit, std::false_type>>( argument_types, result_is_nullable); } else { - return creator_without_type::create< - AggregateFunctionCollect, HasLimit>>( + return creator_without_type::create, HasLimit, std::false_type>>( argument_types, result_is_nullable); } } -template +template AggregateFunctionPtr create_aggregate_function_collect_impl(const std::string& name, const DataTypes& argument_types, const bool result_is_nullable) { @@ -51,25 +59,25 @@ AggregateFunctionPtr create_aggregate_function_collect_impl(const std::string& n } WhichDataType which(remove_nullable(argument_types[0])); -#define DISPATCH(TYPE) \ - if (which.idx == TypeIndex::TYPE) \ - return do_create_agg_function_collect(distinct, argument_types, \ - result_is_nullable); +#define DISPATCH(TYPE) \ + if (which.idx == TypeIndex::TYPE) \ + return do_create_agg_function_collect(distinct, argument_types, \ + result_is_nullable); FOR_NUMERIC_TYPES(DISPATCH) FOR_DECIMAL_TYPES(DISPATCH) #undef DISPATCH if (which.is_date_or_datetime()) { - return do_create_agg_function_collect(distinct, argument_types, - result_is_nullable); + return do_create_agg_function_collect(distinct, argument_types, + result_is_nullable); } else if (which.is_date_v2()) { - return do_create_agg_function_collect(distinct, argument_types, - result_is_nullable); + return do_create_agg_function_collect(distinct, argument_types, + result_is_nullable); } else if (which.is_date_time_v2()) { - return do_create_agg_function_collect(distinct, argument_types, - result_is_nullable); + return do_create_agg_function_collect(distinct, argument_types, + result_is_nullable); } else if (which.is_string()) { - return do_create_agg_function_collect(distinct, argument_types, - result_is_nullable); + return do_create_agg_function_collect( + distinct, argument_types, result_is_nullable); } LOG(WARNING) << fmt::format("unsupported input type {} for aggregate function {}", @@ -81,12 +89,17 @@ AggregateFunctionPtr create_aggregate_function_collect(const std::string& name, const DataTypes& argument_types, const bool result_is_nullable) { if (argument_types.size() == 1) { - return create_aggregate_function_collect_impl(name, argument_types, - result_is_nullable); + if (name == "array_agg") { + return create_aggregate_function_collect_impl( + name, argument_types, result_is_nullable); + } else { + return create_aggregate_function_collect_impl( + name, argument_types, result_is_nullable); + } } if (argument_types.size() == 2) { - return create_aggregate_function_collect_impl(name, argument_types, - result_is_nullable); + return create_aggregate_function_collect_impl( + name, argument_types, result_is_nullable); } LOG(WARNING) << fmt::format("number of parameters for aggregate function {}, should be 1 or 2", name); @@ -96,6 +109,7 @@ AggregateFunctionPtr create_aggregate_function_collect(const std::string& name, void register_aggregate_function_collect_list(AggregateFunctionSimpleFactory& factory) { factory.register_function_both("collect_list", create_aggregate_function_collect); factory.register_function_both("collect_set", create_aggregate_function_collect); + factory.register_function_both("array_agg", create_aggregate_function_collect); factory.register_alias("collect_list", "group_array"); factory.register_alias("collect_set", "group_uniq_array"); } diff --git a/be/src/vec/aggregate_functions/aggregate_function_collect.h b/be/src/vec/aggregate_functions/aggregate_function_collect.h index 4e774b887e6aafe..78aa01ac68268ba 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_collect.h +++ b/be/src/vec/aggregate_functions/aggregate_function_collect.h @@ -288,30 +288,173 @@ struct AggregateFunctionCollectListData { } }; -template +template +struct AggregateFunctionArrayAggData { + using ElementType = T; + using ColVecType = ColumnVectorOrDecimal; + MutableColumnPtr column_data; + ColVecType* nested_column; + NullMap* null_map; + + AggregateFunctionArrayAggData(const DataTypes& argument_types) { + if constexpr (IsDecimalNumber) { + DataTypePtr column_type = make_nullable(argument_types[0]); + column_data = column_type->create_column(); + null_map = &(assert_cast(*column_data).get_null_map_data()); + nested_column = assert_cast( + assert_cast(*column_data).get_nested_column_ptr().get()); + } + } + + AggregateFunctionArrayAggData() { + if constexpr (!IsDecimalNumber) { + column_data = ColumnNullable::create(ColVecType::create(), ColumnUInt8::create()); + null_map = &(assert_cast(*column_data).get_null_map_data()); + nested_column = assert_cast( + assert_cast(*column_data).get_nested_column_ptr().get()); + } + } + + void add(const IColumn& column, size_t row_num) { + const auto& col = assert_cast(column); + const auto& vec = assert_cast(col.get_nested_column()).get_data(); + null_map->push_back(col.get_null_map_data()[row_num]); + nested_column->get_data().push_back(vec[row_num]); + DCHECK(null_map->size() == nested_column->size()); + } + + void deserialize_and_merge(const IColumn& column, size_t row_num) { + auto& to_arr = assert_cast(column); + auto& to_nested_col = to_arr.get_data(); + auto col_null = reinterpret_cast(&to_nested_col); + const auto& vec = assert_cast(col_null->get_nested_column()).get_data(); + auto start = to_arr.get_offsets()[row_num - 1]; + auto end = start + to_arr.get_offsets()[row_num] - to_arr.get_offsets()[row_num - 1]; + for (auto i = start; i < end; ++i) { + null_map->push_back(col_null->get_null_map_data()[i]); + nested_column->get_data().push_back(vec[i]); + } + } + + void reset() { + null_map->clear(); + nested_column->clear(); + } + + void insert_result_into(IColumn& to) const { + auto& to_arr = assert_cast(to); + auto& to_nested_col = to_arr.get_data(); + auto col_null = reinterpret_cast(&to_nested_col); + auto& vec = assert_cast(col_null->get_nested_column()).get_data(); + size_t num_rows = null_map->size(); + auto& nested_column_data = nested_column->get_data(); + for (size_t i = 0; i < num_rows; ++i) { + col_null->get_null_map_data().push_back((*null_map)[i]); + vec.push_back(nested_column_data[i]); + } + to_arr.get_offsets().push_back(to_nested_col.size()); + } +}; + +template <> +struct AggregateFunctionArrayAggData { + using ElementType = StringRef; + using ColVecType = ColumnString; + MutableColumnPtr column_data; + ColVecType* nested_column; + NullMap* null_map; + + AggregateFunctionArrayAggData() { + column_data = ColumnNullable::create(ColVecType::create(), ColumnUInt8::create()); + null_map = &(assert_cast(*column_data).get_null_map_data()); + nested_column = assert_cast( + assert_cast(*column_data).get_nested_column_ptr().get()); + } + + void add(const IColumn& column, size_t row_num) { + const auto& col = assert_cast(column); + const auto& vec = assert_cast(col.get_nested_column()); + null_map->push_back(col.get_null_map_data()[row_num]); + nested_column->insert_from(vec, row_num); + DCHECK(null_map->size() == nested_column->size()); + } + + void deserialize_and_merge(const IColumn& column, size_t row_num) { + auto& to_arr = assert_cast(column); + auto& to_nested_col = to_arr.get_data(); + auto col_null = reinterpret_cast(&to_nested_col); + const auto& vec = assert_cast(col_null->get_nested_column()); + auto start = to_arr.get_offsets()[row_num - 1]; + auto end = start + to_arr.get_offsets()[row_num] - to_arr.get_offsets()[row_num - 1]; + for (auto i = start; i < end; ++i) { + null_map->push_back(col_null->get_null_map_data()[i]); + nested_column->insert_from(vec, i); + } + } + + void reset() { + null_map->clear(); + nested_column->clear(); + } + + void insert_result_into(IColumn& to) const { + auto& to_arr = assert_cast(to); + auto& to_nested_col = to_arr.get_data(); + auto col_null = reinterpret_cast(&to_nested_col); + auto& vec = assert_cast(col_null->get_nested_column()); + size_t num_rows = null_map->size(); + for (size_t i = 0; i < num_rows; ++i) { + col_null->get_null_map_data().push_back((*null_map)[i]); + vec.insert_from(*nested_column, i); + } + to_arr.get_offsets().push_back(to_nested_col.size()); + } +}; + +//ShowNull is just used to support array_agg because array_agg needs to display NULL +//todo: Supports order by sorting for array_agg +template class AggregateFunctionCollect - : public IAggregateFunctionDataHelper> { + : public IAggregateFunctionDataHelper> { using GenericType = AggregateFunctionCollectSetData; static constexpr bool ENABLE_ARENA = std::is_same_v; public: + using BaseHelper = IAggregateFunctionHelper>; + AggregateFunctionCollect(const DataTypes& argument_types, UInt64 max_size_ = std::numeric_limits::max()) - : IAggregateFunctionDataHelper>( + : IAggregateFunctionDataHelper>( {argument_types}), return_type(argument_types[0]) {} std::string get_name() const override { - if constexpr (std::is_same_v, - Data>) { + if constexpr (ShowNull::value) { + return "array_agg"; + } else if constexpr (std::is_same_v, + Data>) { return "collect_list"; } else { return "collect_set"; } } + void create(AggregateDataPtr __restrict place) const override { + if constexpr (ShowNull::value) { + if constexpr (IsDecimalNumber) { + new (place) Data(argument_types); + } else { + new (place) Data(); + } + } else { + new (place) Data(); + } + } + DataTypePtr get_return_type() const override { return std::make_shared(make_nullable(return_type)); } @@ -343,35 +486,181 @@ class AggregateFunctionCollect auto& rhs_data = this->data(rhs); if constexpr (ENABLE_ARENA) { data.merge(rhs_data, arena); - } else { + } else if constexpr (!ShowNull::value) { data.merge(rhs_data); } } void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& buf) const override { - this->data(place).write(buf); + if constexpr (!ShowNull::value) { + this->data(place).write(buf); + } } void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena*) const override { - this->data(place).read(buf); + if constexpr (!ShowNull::value) { + this->data(place).read(buf); + } } void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const override { auto& to_arr = assert_cast(to); auto& to_nested_col = to_arr.get_data(); - if (to_nested_col.is_nullable()) { + if constexpr (ShowNull::value) { + DCHECK(to_nested_col.is_nullable()); + this->data(place).insert_result_into(to); + } else { + if (to_nested_col.is_nullable()) { + auto col_null = reinterpret_cast(&to_nested_col); + this->data(place).insert_result_into(col_null->get_nested_column()); + col_null->get_null_map_data().resize_fill(col_null->get_nested_column().size(), 0); + } else { + this->data(place).insert_result_into(to_nested_col); + } + to_arr.get_offsets().push_back(to_nested_col.size()); + } + } + + void serialize_without_key_to_column(ConstAggregateDataPtr __restrict place, + IColumn& to) const override { + if constexpr (ShowNull::value) { + this->data(place).insert_result_into(to); + } else { + return BaseHelper::serialize_without_key_to_column(place, to); + } + } + + void deserialize_and_merge_from_column(AggregateDataPtr __restrict place, const IColumn& column, + Arena* arena) const override { + if constexpr (ShowNull::value) { + const size_t num_rows = column.size(); + for (size_t i = 0; i != num_rows; ++i) { + this->data(place).deserialize_and_merge(column, i); + } + } else { + return BaseHelper::deserialize_and_merge_from_column(place, column, arena); + } + } + + void deserialize_and_merge_vec(const AggregateDataPtr* places, size_t offset, + AggregateDataPtr rhs, const ColumnString* column, Arena* arena, + const size_t num_rows) const override { + if constexpr (ShowNull::value) { + for (size_t i = 0; i != num_rows; ++i) { + this->data(places[i]).deserialize_and_merge(*assert_cast(column), + i); + } + } else { + return BaseHelper::deserialize_and_merge_vec(places, offset, rhs, column, arena, + num_rows); + } + } + + void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena* arena, + size_t num_rows) const override { + if constexpr (ShowNull::value) { + for (size_t i = 0; i != num_rows; ++i) { + this->data(places).deserialize_and_merge(column, i); + } + } else { + return BaseHelper::deserialize_from_column(places, column, arena, num_rows); + } + } + + void deserialize_and_merge_from_column_range(AggregateDataPtr __restrict place, + const IColumn& column, size_t begin, size_t end, + Arena* arena) const override { + if constexpr (ShowNull::value) { + DCHECK(end <= column.size() && begin <= end) << ", begin:" << begin << ", end:" << end + << ", column.size():" << column.size(); + for (size_t i = begin; i <= end; ++i) { + this->data(place).deserialize_and_merge(column, i); + } + } else { + return BaseHelper::deserialize_and_merge_from_column_range(place, column, begin, end, + arena); + } + } + + void deserialize_and_merge_vec_selected(const AggregateDataPtr* places, size_t offset, + AggregateDataPtr rhs, const ColumnString* column, + Arena* arena, const size_t num_rows) const override { + if constexpr (ShowNull::value) { + for (size_t i = 0; i != num_rows; ++i) { + if (places[i]) { + this->data(places[i]).deserialize_and_merge( + *assert_cast(column), i); + } + } + } else { + return BaseHelper::deserialize_and_merge_vec_selected(places, offset, rhs, column, + arena, num_rows); + } + } + + void serialize_to_column(const std::vector& places, size_t offset, + MutableColumnPtr& dst, const size_t num_rows) const override { + if constexpr (ShowNull::value) { + for (size_t i = 0; i != num_rows; ++i) { + Data& data_ = this->data(places[i] + offset); + data_.insert_result_into(*dst); + } + } else { + return BaseHelper::serialize_to_column(places, offset, dst, num_rows); + } + } + + void streaming_agg_serialize_to_column(const IColumn** columns, MutableColumnPtr& dst, + const size_t num_rows, Arena* arena) const override { + if constexpr (ShowNull::value) { + auto& to_arr = assert_cast(*dst); + auto& to_nested_col = to_arr.get_data(); + DCHECK(num_rows == columns[0]->size()); auto col_null = reinterpret_cast(&to_nested_col); - this->data(place).insert_result_into(col_null->get_nested_column()); - col_null->get_null_map_data().resize_fill(col_null->get_nested_column().size(), 0); + const auto& col_src = assert_cast(*(columns[0])); + + for (size_t i = 0; i < num_rows; ++i) { + col_null->get_null_map_data().push_back(col_src.get_null_map_data()[i]); + if constexpr (std::is_same_v) { + auto& vec = assert_cast(col_null->get_nested_column()); + const auto& vec_src = + assert_cast(col_src.get_nested_column()); + vec.insert_from(vec_src, i); + } else { + using ColVecType = ColumnVectorOrDecimal; + auto& vec = assert_cast(col_null->get_nested_column()).get_data(); + auto& vec_src = + assert_cast(col_src.get_nested_column()).get_data(); + vec.push_back(vec_src[i]); + } + to_arr.get_offsets().push_back(to_nested_col.size()); + } + } else { - this->data(place).insert_result_into(to_nested_col); + return BaseHelper::streaming_agg_serialize_to_column(columns, dst, num_rows, arena); + } + } + + [[nodiscard]] MutableColumnPtr create_serialize_column() const override { + if constexpr (ShowNull::value) { + return get_return_type()->create_column(); + } else { + return ColumnString::create(); + } + } + + [[nodiscard]] DataTypePtr get_serialized_type() const override { + if constexpr (ShowNull::value) { + return std::make_shared(make_nullable(return_type)); + } else { + return IAggregateFunction::get_serialized_type(); } - to_arr.get_offsets().push_back(to_nested_col.size()); } private: DataTypePtr return_type; + using IAggregateFunction::argument_types; }; } // namespace doris::vectorized \ No newline at end of file diff --git a/docs/en/docs/sql-manual/sql-functions/aggregate-functions/array-agg.md b/docs/en/docs/sql-manual/sql-functions/aggregate-functions/array-agg.md new file mode 100644 index 000000000000000..de61da28628eabd --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/aggregate-functions/array-agg.md @@ -0,0 +1,84 @@ +--- +{ + "title": "ARRAY_AGG", + "language": "en" +} +--- + + + +## ARRAY_AGG + +### description + +#### Syntax + +`ARRAY_AGG(col)` + +Concatenation of values in a column (including the null value) into an array can be used for multiple rows to one row (row to column). + +### notice + +- The order of the elements in an array is not guaranteed. +- Returns the array generated by the transformation. The element type in the array is the same as the col type. + +### example + +```sql +mysql> select * from test_doris_array_agg; + ++------+------+ + +| c1 | c2 | + ++------+------+ + +| 1 | a | + +| 1 | b | + +| 2 | c | + +| 2 | NULL | + +| 3 | NULL | + ++------+------+ + +mysql> select c1, array_agg(c2) from test_doris_array_agg group by c1; + ++------+-----------------+ + +| c1 | array_agg(`c2`) | + ++------+-----------------+ + +| 1 | ["a","b"] | + +| 2 | [NULL,"c"] | + +| 3 | [NULL] | + ++------+-----------------+ +``` + +### keywords + +ARRAY_AGG diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/aggregate-functions/array-agg.md b/docs/zh-CN/docs/sql-manual/sql-functions/aggregate-functions/array-agg.md new file mode 100644 index 000000000000000..34549ab059e6a1e --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/aggregate-functions/array-agg.md @@ -0,0 +1,84 @@ +--- +{ + "title": "ARRAY_AGG", + "language": "zh-CN" +} +--- + + + +## ARRAY_AGG + +### description + +#### Syntax + +`ARRAY_AGG(col)` + +将一列中的值(包括空值 null)串联成一个数组,可以用于多行转一行(行转列)。 + +### notice + +- 数组中元素不保证顺序。 +- 返回转换生成的数组。数组中的元素类型与 `col` 类型一致。 + +### example + +```sql +mysql> select * from test_doris_array_agg; + ++------+------+ + +| c1 | c2 | + ++------+------+ + +| 1 | a | + +| 1 | b | + +| 2 | c | + +| 2 | NULL | + +| 3 | NULL | + ++------+------+ + +mysql> select c1, array_agg(c2) from test_doris_array_agg group by c1; + ++------+-----------------+ + +| c1 | array_agg(`c2`) | + ++------+-----------------+ + +| 1 | ["a","b"] | + +| 2 | [NULL,"c"] | + +| 3 | [NULL] | + ++------+-----------------+ +``` + +### keywords + +ARRAY_AGG diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java index 53e3458c3e7b02c..855f2036d82e12c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java @@ -1566,7 +1566,8 @@ && collectChildReturnTypes()[0].isDecimalV3()) { } if (fnName.getFunction().equalsIgnoreCase("collect_list") - || fnName.getFunction().equalsIgnoreCase("collect_set")) { + || fnName.getFunction().equalsIgnoreCase("collect_set") + || fnName.getFunction().equalsIgnoreCase("array_agg")) { fn.setReturnType(new ArrayType(getChild(0).type)); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/AggregateFunction.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/AggregateFunction.java index b646c7ef982946a..2dceb302b7410a4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/AggregateFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/AggregateFunction.java @@ -54,7 +54,8 @@ public class AggregateFunction extends Function { FunctionSet.INTERSECT_COUNT, FunctionSet.ORTHOGONAL_BITMAP_UNION_COUNT, FunctionSet.COUNT, "approx_count_distinct", "ndv", FunctionSet.BITMAP_UNION_INT, FunctionSet.BITMAP_UNION_COUNT, "ndv_no_finalize", FunctionSet.WINDOW_FUNNEL, FunctionSet.RETENTION, - FunctionSet.SEQUENCE_MATCH, FunctionSet.SEQUENCE_COUNT, FunctionSet.MAP_AGG, FunctionSet.BITMAP_AGG); + FunctionSet.SEQUENCE_MATCH, FunctionSet.SEQUENCE_COUNT, FunctionSet.MAP_AGG, FunctionSet.BITMAP_AGG, + FunctionSet.ARRAY_AGG); public static ImmutableSet ALWAYS_NULLABLE_AGGREGATE_FUNCTION_NAME_SET = ImmutableSet.of("stddev_samp", "variance_samp", "var_samp", "percentile_approx"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java index cde3cecf17006bc..e1c62cc739d2b73 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java @@ -572,6 +572,8 @@ public void addBuiltinBothScalaAndVectorized(Function fn) { public static final String GROUP_ARRAY = "group_array"; + public static final String ARRAY_AGG = "array_agg"; + // Populate all the aggregate builtins in the catalog. // null symbols indicate the function does not need that step of the evaluation. // An empty symbol indicates a TODO for the BE to implement the function. @@ -1402,6 +1404,9 @@ private void initAggregateBuiltins() { AggregateFunction.createBuiltin(GROUP_ARRAY, Lists.newArrayList(t, Type.INT), new ArrayType(t), t, "", "", "", "", "", true, false, true, true)); + addBuiltin(AggregateFunction.createBuiltin(ARRAY_AGG, Lists.newArrayList(t), new ArrayType(t), t, "", "", "", "", "", + true, false, true, true)); + //first_value/last_value for array addBuiltin(AggregateFunction.createAnalyticBuiltin("first_value", Lists.newArrayList(new ArrayType(t)), new ArrayType(t), Type.ARRAY, diff --git a/regression-test/data/query_p0/aggregate/array_agg.out b/regression-test/data/query_p0/aggregate/array_agg.out new file mode 100644 index 000000000000000..ee7260205a4c159 --- /dev/null +++ b/regression-test/data/query_p0/aggregate/array_agg.out @@ -0,0 +1,67 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql1 -- +["LC", "LB", "alex"] +["LC", "LB", "LA"] +["LC", NULL, "LA"] +["LC", "LB", "LA"] +[NULL, "LC", "LB", "LA"] +[NULL, "LC", "LC", "LC", "LC"] +[NULL, "LC", "LC", "LC", "LC"] + +-- !sql2 -- +["alex", NULL, "LC", "LC", "LC", "LC"] +["LB"] +["LC"] +["LA"] +["LB"] +["LC"] +["LA"] +["LC"] +["LA"] +["LB"] +["LC"] +["LA"] +["LB"] +[NULL, "LC"] +[NULL, "LC", "LC"] +[NULL, "LC", "LC"] + +-- !sql3 -- +["LC", "LB", "alex", "LC", "LB", "LA", "LC", NULL, "LA", "LC", "LB", "LA", NULL, "LC", "LB", "LA", NULL, "LC", "LC", "LC", "LC", NULL, "LC", "LC", "LC", "LC"] + +-- !sql4 -- +["V1_3", "V1_2", NULL, "V2_3", "V2_2", "V2_1", "V3_3", NULL, "V3_1", "V4_3", "V4_2", "V4_1", "V5_3", "V5_3", "V5_2", "V5_1", "V6_3", NULL, "V6_3", NULL, "V6_3", "V7_3", NULL, "V7_3", NULL, "V7_3"] + +-- !sql5 -- +1 [2, 1, NULL] +2 [5, 5, 4] +3 [NULL, 6, 6] +4 [6, 6, 5] +5 [NULL, 5, 6] +6 [NULL, NULL, NULL, NULL, NULL] +7 [NULL, NULL, NULL, NULL, NULL] + +-- !sql6 -- +[""] + +-- !sql7 -- +["LC", "LB", "alex"] +["LC", "LB", "LA"] +["LC", NULL, "LA"] +["LC", "LB", "LA"] +["LC", "LB", "LA"] +[""] +[""] + +-- !sql8 -- +[NULL] + +-- !sql9 -- +1 [1.11, NULL, NULL] +2 [1.21, NULL, 1.23] +3 [1.24, 1.23, 1.21] +4 [1.22, NULL, 1.22] +5 [NULL, NULL, NULL] +7 [NULL] +8 [NULL] + diff --git a/regression-test/suites/query_p0/aggregate/array_agg.groovy b/regression-test/suites/query_p0/aggregate/array_agg.groovy new file mode 100644 index 000000000000000..1463eed6470cb84 --- /dev/null +++ b/regression-test/suites/query_p0/aggregate/array_agg.groovy @@ -0,0 +1,195 @@ +// 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("array_agg") { + sql "DROP TABLE IF EXISTS `test_array_agg`;" + sql "DROP TABLE IF EXISTS `test_array_agg_int`;" + sql "DROP TABLE IF EXISTS `test_array_agg_decimal`;" + sql """ + CREATE TABLE `test_array_agg` ( + `id` int(11) NOT NULL, + `label_name` varchar(32) default null, + `value_field` string default null, + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false" + ); + """ + + sql """ + CREATE TABLE `test_array_agg_int` ( + `id` int(11) NOT NULL, + `label_name` varchar(32) default null, + `value_field` string default null, + `age` int(11) default null + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false" + ); + """ + + sql """ + CREATE TABLE `test_array_agg_decimal` ( + `id` int(11) NOT NULL, + `label_name` varchar(32) default null, + `value_field` string default null, + `age` int(11) default null, + `o_totalprice` DECIMAL(15, 2) default NULL, + `label_name_not_null` varchar(32) not null + )ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false" + ); + """ + + sql """ + insert into `test_array_agg` values + (1, "alex",NULL), + (1, "LB", "V1_2"), + (1, "LC", "V1_3"), + (2, "LA", "V2_1"), + (2, "LB", "V2_2"), + (2, "LC", "V2_3"), + (3, "LA", "V3_1"), + (3, NULL, NULL), + (3, "LC", "V3_3"), + (4, "LA", "V4_1"), + (4, "LB", "V4_2"), + (4, "LC", "V4_3"), + (5, "LA", "V5_1"), + (5, "LB", "V5_2"), + (5, "LC", "V5_3"), + (5, NULL, "V5_3"), + (6, "LC", "V6_3"), + (6, "LC", NULL), + (6, "LC", "V6_3"), + (6, "LC", NULL), + (6, NULL, "V6_3"), + (7, "LC", "V7_3"), + (7, "LC", NULL), + (7, "LC", "V7_3"), + (7, "LC", NULL), + (7, NULL, "V7_3"); + """ + + sql """ + insert into `test_array_agg_int` values + (1, "alex",NULL,NULL), + (1, "LB", "V1_2",1), + (1, "LC", "V1_3",2), + (2, "LA", "V2_1",4), + (2, "LB", "V2_2",5), + (2, "LC", "V2_3",5), + (3, "LA", "V3_1",6), + (3, NULL, NULL,6), + (3, "LC", "V3_3",NULL), + (4, "LA", "V4_1",5), + (4, "LB", "V4_2",6), + (4, "LC", "V4_3",6), + (5, "LA", "V5_1",6), + (5, "LB", "V5_2",5), + (5, "LC", "V5_3",NULL), + (6, "LC", "V6_3",NULL), + (6, "LC", NULL,NULL), + (6, "LC", "V6_3",NULL), + (6, "LC", NULL,NULL), + (6, NULL, "V6_3",NULL), + (7, "LC", "V7_3",NULL), + (7, "LC", NULL,NULL), + (7, "LC", "V7_3",NULL), + (7, "LC", NULL,NULL), + (7, NULL, "V7_3",NULL); + """ + + sql """ + insert into `test_array_agg_decimal` values + (1, "alex",NULL,NULL,NULL,"alex"), + (1, "LB", "V1_2",1,NULL,"alexxing"), + (1, "LC", "V1_3",2,1.11,"alexcoco"), + (2, "LA", "V2_1",4,1.23,"alex662"), + (2, "LB", "",5,NULL,""), + (2, "LC", "",5,1.21,"alexcoco1"), + (3, "LA", "V3_1",6,1.21,"alexcoco2"), + (3, NULL, NULL,6,1.23,"alexcoco3"), + (3, "LC", "V3_3",NULL,1.24,"alexcoco662"), + (4, "LA", "",5,1.22,"alexcoco662"), + (4, "LB", "V4_2",6,NULL,"alexcoco662"), + (4, "LC", "V4_3",6,1.22,"alexcoco662"), + (5, "LA", "V5_1",6,NULL,"alexcoco662"), + (5, "LB", "V5_2",5,NULL,"alexcoco662"), + (5, "LC", "V5_3",NULL,NULL,"alexcoco662"), + (7, "", NULL,NULL,NULL,"alexcoco1"), + (8, "", NULL,0,NULL,"alexcoco2"); + """ + + qt_sql1 """ + SELECT array_agg(`label_name`) FROM `test_array_agg` GROUP BY `id` order by id; + """ + qt_sql2 """ + SELECT array_agg(label_name) FROM `test_array_agg` GROUP BY value_field order by value_field; + """ + qt_sql3 """ + SELECT array_agg(`label_name`) FROM `test_array_agg`; + """ + qt_sql4 """ + SELECT array_agg(`value_field`) FROM `test_array_agg`; + """ + qt_sql5 """ + SELECT id, array_agg(age) FROM test_array_agg_int GROUP BY id order by id; + """ + + qt_sql6 """ + select array_agg(label_name) from test_array_agg_decimal where id=7; + """ + + qt_sql7 """ + select array_agg(label_name) from test_array_agg_decimal group by id order by id; + """ + + qt_sql8 """ + select array_agg(age) from test_array_agg_decimal where id=7; + """ + + qt_sql9 """ + select id,array_agg(o_totalprice) from test_array_agg_decimal group by id order by id; + """ + + sql "DROP TABLE `test_array_agg`" + sql "DROP TABLE `test_array_agg_int`" + sql "DROP TABLE `test_array_agg_decimal`" +} From 4144e62c5fb159274024d70fb31292f0270b62de Mon Sep 17 00:00:00 2001 From: HappenLee Date: Thu, 7 Sep 2023 10:22:05 +0800 Subject: [PATCH 06/14] [pipeline](log) disable some unless error status warning log (#23961) --- be/src/common/status.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/be/src/common/status.h b/be/src/common/status.h index 6abff8c49961d16..44c840a2d096f50 100644 --- a/be/src/common/status.h +++ b/be/src/common/status.h @@ -313,7 +313,9 @@ constexpr bool capture_stacktrace(int code) { && code != ErrorCode::KEY_NOT_FOUND && code != ErrorCode::KEY_ALREADY_EXISTS && code != ErrorCode::CANCELLED - && code != ErrorCode::UNINITIALIZED; + && code != ErrorCode::UNINITIALIZED + && code != ErrorCode::PIP_WAIT_FOR_RF + && code != ErrorCode::PIP_WAIT_FOR_SC; } // clang-format on From 3acf0261724af7add13ac52269d19eac2e192f18 Mon Sep 17 00:00:00 2001 From: airborne12 Date: Thu, 7 Sep 2023 10:24:27 +0800 Subject: [PATCH 07/14] [Enhancement](inverted index) add bkd index query cache to improve perf (#23952) use query cache to accelerate bkd query performance, especially for high concurrency. --- .../rowset/segment_v2/inverted_index_cache.h | 5 +- .../segment_v2/inverted_index_reader.cpp | 105 +++++++++++------- .../rowset/segment_v2/inverted_index_reader.h | 13 ++- 3 files changed, 77 insertions(+), 46 deletions(-) diff --git a/be/src/olap/rowset/segment_v2/inverted_index_cache.h b/be/src/olap/rowset/segment_v2/inverted_index_cache.h index 388ee02ee96aa85..c67e17ddda968e0 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_cache.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_cache.h @@ -199,7 +199,7 @@ class InvertedIndexQueryCache : public LRUCachePolicy { io::Path index_path; // index file path std::string column_name; // column name InvertedIndexQueryType query_type; // query type - std::wstring value; // query value + std::string value; // query value // Encode to a flat binary which can be used as LRUCache's key std::string encode() const { @@ -213,8 +213,7 @@ class InvertedIndexQueryCache : public LRUCachePolicy { } key_buf.append(query_type_str); key_buf.append("/"); - auto str = lucene_wcstoutf8string(value.c_str(), value.length()); - key_buf.append(str); + key_buf.append(value); return key_buf; } }; diff --git a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp index c2a622699a51be6..a521da394c79992 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp @@ -53,7 +53,6 @@ #include "io/fs/file_system.h" #include "olap/key_coder.h" #include "olap/olap_common.h" -#include "olap/rowset/segment_v2/inverted_index_cache.h" #include "olap/rowset/segment_v2/inverted_index_compound_directory.h" #include "olap/rowset/segment_v2/inverted_index_desc.h" #include "olap/types.h" @@ -178,7 +177,7 @@ Status InvertedIndexReader::read_null_bitmap(InvertedIndexQueryCacheHandle* cach _index_meta.index_id()); auto index_file_path = index_dir / index_file_name; InvertedIndexQueryCache::CacheKey cache_key { - index_file_path, "", InvertedIndexQueryType::UNKNOWN_QUERY, L"null_bitmap"}; + index_file_path, "", InvertedIndexQueryType::UNKNOWN_QUERY, "null_bitmap"}; auto cache = InvertedIndexQueryCache::instance(); if (cache->lookup(cache_key, cache_handle)) { return Status::OK(); @@ -331,9 +330,9 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, const std::string roaring::Roaring query_match_bitmap; bool null_bitmap_already_read = false; if (query_type == InvertedIndexQueryType::MATCH_PHRASE_QUERY) { - std::wstring str_tokens; + std::wstring wstr_tokens; for (auto& token : analyse_result) { - str_tokens += token; + wstr_tokens += token; } auto cache = InvertedIndexQueryCache::instance(); @@ -341,6 +340,7 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, const std::string cache_key.index_path = index_file_path; cache_key.column_name = column_name; cache_key.query_type = InvertedIndexQueryType::MATCH_PHRASE_QUERY; + auto str_tokens = lucene_wcstoutf8string(wstr_tokens.c_str(), wstr_tokens.length()); cache_key.value.swap(str_tokens); InvertedIndexQueryCacheHandle cache_handle; std::shared_ptr term_match_bitmap = nullptr; @@ -375,9 +375,10 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, const std::string // try to get term bitmap match result from cache to avoid query index on cache hit auto cache = InvertedIndexQueryCache::instance(); // use EQUAL_QUERY type here since cache is for each term/token - InvertedIndexQueryCache::CacheKey cache_key {index_file_path, column_name, - InvertedIndexQueryType::EQUAL_QUERY, - token_ws}; + auto token = lucene_wcstoutf8string(token_ws.c_str(), token_ws.length()); + + InvertedIndexQueryCache::CacheKey cache_key { + index_file_path, column_name, InvertedIndexQueryType::EQUAL_QUERY, token}; VLOG_DEBUG << "cache_key:" << cache_key.encode(); InvertedIndexQueryCacheHandle cache_handle; if (cache->lookup(cache_key, &cache_handle)) { @@ -474,7 +475,7 @@ Status StringTypeInvertedIndexReader::query(OlapReaderStatistics* stats, // try to get query bitmap result from cache and return immediately on cache hit InvertedIndexQueryCache::CacheKey cache_key {index_file_path, column_name, query_type, - search_str_ws}; + search_str}; auto cache = InvertedIndexQueryCache::instance(); InvertedIndexQueryCacheHandle cache_handle; if (cache->lookup(cache_key, &cache_handle)) { @@ -593,6 +594,7 @@ BkdIndexReader::BkdIndexReader(io::FileSystemSPtr fs, const std::string& path, LOG(WARNING) << "bkd index: " << index_file.string() << " not exist."; return; } + _file_full_path = index_file; _compoundReader = std::make_unique( DorisCompoundDirectory::getDirectory(fs, index_dir.c_str()), index_file_name.c_str(), config::inverted_index_read_buffer_size); @@ -606,9 +608,8 @@ Status BkdIndexReader::new_iterator(OlapReaderStatistics* stats, Status BkdIndexReader::bkd_query(OlapReaderStatistics* stats, const std::string& column_name, const void* query_value, InvertedIndexQueryType query_type, - std::shared_ptr& r, + std::shared_ptr r, InvertedIndexVisitor* visitor) { - RETURN_IF_ERROR(get_bkd_reader(r)); char tmp[r->bytes_per_dim_]; switch (query_type) { case InvertedIndexQueryType::EQUAL_QUERY: { @@ -643,6 +644,20 @@ Status BkdIndexReader::try_query(OlapReaderStatistics* stats, const std::string& uint32_t* count) { auto visitor = std::make_unique(nullptr, query_type, true); std::shared_ptr r; + RETURN_IF_ERROR(get_bkd_reader(&r)); + std::string query_str; + _value_key_coder->full_encode_ascending(query_value, &query_str); + + InvertedIndexQueryCache::CacheKey cache_key {_file_full_path, column_name, query_type, + query_str}; + auto cache = InvertedIndexQueryCache::instance(); + InvertedIndexQueryCacheHandle cache_handler; + roaring::Roaring bit_map; + auto cache_status = handle_cache(cache, cache_key, &cache_handler, stats, &bit_map); + if (cache_status.ok()) { + *count = bit_map.cardinality(); + return Status::OK(); + } try { auto st = bkd_query(stats, column_name, query_value, query_type, r, visitor.get()); if (!st.ok()) { @@ -662,34 +677,42 @@ Status BkdIndexReader::try_query(OlapReaderStatistics* stats, const std::string& return Status::OK(); } +Status BkdIndexReader::handle_cache(InvertedIndexQueryCache* cache, + const InvertedIndexQueryCache::CacheKey& cache_key, + InvertedIndexQueryCacheHandle* cache_handler, + OlapReaderStatistics* stats, roaring::Roaring* bit_map) { + if (cache->lookup(cache_key, cache_handler)) { + stats->inverted_index_query_cache_hit++; + SCOPED_RAW_TIMER(&stats->inverted_index_query_bitmap_copy_timer); + *bit_map = *cache_handler->get_bitmap(); + return Status::OK(); + } else { + stats->inverted_index_query_cache_miss++; + return Status::Error("cache miss"); + } +} + Status BkdIndexReader::query(OlapReaderStatistics* stats, const std::string& column_name, const void* query_value, InvertedIndexQueryType query_type, roaring::Roaring* bit_map) { SCOPED_RAW_TIMER(&stats->inverted_index_query_timer); - io::Path path(_path); - auto index_dir = path.parent_path(); - auto index_file_name = - InvertedIndexDescriptor::get_index_file_name(path.filename(), _index_meta.index_id()); - auto index_file_path = index_dir / index_file_name; - // std::string query_str {(const char *)query_value}; - - // // try to get query bitmap result from cache and return immediately on cache hit - // InvertedIndexQueryCache::CacheKey cache_key - // {index_file_path, column_name, query_type, std::wstring(query_str.begin(), query_str.end())}; - // auto cache = InvertedIndexQueryCache::instance(); - // InvertedIndexQueryCacheHandle cache_handle; - // if (cache->lookup(cache_key, &cache_handle)) { - // stats->inverted_index_query_cache_hit++; - // SCOPED_RAW_TIMER(&stats->inverted_index_query_bitmap_copy_timer); - // *bit_map = *cache_handle.match_bitmap(); - // return Status::OK(); - // } else { - // stats->inverted_index_query_cache_miss++; - // } - auto visitor = std::make_unique(bit_map, query_type); std::shared_ptr r; + RETURN_IF_ERROR(get_bkd_reader(&r)); + + std::string query_str; + _value_key_coder->full_encode_ascending(query_value, &query_str); + + InvertedIndexQueryCache::CacheKey cache_key {_file_full_path, column_name, query_type, + query_str}; + auto cache = InvertedIndexQueryCache::instance(); + InvertedIndexQueryCacheHandle cache_handler; + auto cache_status = handle_cache(cache, cache_key, &cache_handler, stats, bit_map); + if (cache_status.ok()) { + return Status::OK(); + } + try { auto st = bkd_query(stats, column_name, query_value, query_type, r, visitor.get()); if (!st.ok()) { @@ -705,17 +728,17 @@ Status BkdIndexReader::query(OlapReaderStatistics* stats, const std::string& col "BKD Query CLuceneError Occurred, error msg: {}", e.what()); } - // // add to cache - // roaring::Roaring* term_match_bitmap = new roaring::Roaring(*bit_map); - // term_match_bitmap->runOptimize(); - // cache->insert(cache_key, term_match_bitmap, &cache_handle); + std::shared_ptr query_bitmap = std::make_shared(*bit_map); + query_bitmap->runOptimize(); + cache->insert(cache_key, query_bitmap, &cache_handler); VLOG_DEBUG << "BKD index search column: " << column_name << " result: " << bit_map->cardinality(); + return Status::OK(); } -Status BkdIndexReader::get_bkd_reader(std::shared_ptr& bkdReader) { +Status BkdIndexReader::get_bkd_reader(std::shared_ptr* bkdReader) { // bkd file reader if (_compoundReader == nullptr) { return Status::Error( @@ -739,18 +762,18 @@ Status BkdIndexReader::get_bkd_reader(std::shared_ptr(data_in.release()); - if (0 == bkdReader->read_meta(meta_in.get())) { + *bkdReader = std::make_shared(data_in.release()); + if (0 == (*bkdReader)->read_meta(meta_in.get())) { VLOG_NOTICE << "bkd index file is empty:" << _compoundReader->toString(); return Status::EndOfFile("bkd index file is empty"); } - bkdReader->read_index(index_in.get()); + (*bkdReader)->read_index(index_in.get()); - _type_info = get_scalar_type_info((FieldType)bkdReader->type); + _type_info = get_scalar_type_info((FieldType)(*bkdReader)->type); if (_type_info == nullptr) { return Status::Error( - "unsupported typeinfo, type={}", bkdReader->type); + "unsupported typeinfo, type={}", (*bkdReader)->type); } _value_key_coder = get_key_coder(_type_info->type()); return Status::OK(); diff --git a/be/src/olap/rowset/segment_v2/inverted_index_reader.h b/be/src/olap/rowset/segment_v2/inverted_index_reader.h index c076756805d4829..5f7b318825eaae8 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_reader.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_reader.h @@ -29,6 +29,7 @@ #include "io/fs/file_system.h" #include "io/fs/path.h" #include "olap/inverted_index_parser.h" +#include "olap/rowset/segment_v2/inverted_index_cache.h" #include "olap/rowset/segment_v2/inverted_index_compound_reader.h" #include "olap/rowset/segment_v2/inverted_index_query_type.h" #include "olap/tablet_schema.h" @@ -192,6 +193,9 @@ class InvertedIndexVisitor : public lucene::util::bkd::bkd_reader::intersect_vis class BkdIndexReader : public InvertedIndexReader { ENABLE_FACTORY_CREATOR(BkdIndexReader); +private: + std::string _file_full_path; + public: explicit BkdIndexReader(io::FileSystemSPtr fs, const std::string& path, const TabletIndex* index_meta); @@ -221,11 +225,16 @@ class BkdIndexReader : public InvertedIndexReader { uint32_t* count) override; Status bkd_query(OlapReaderStatistics* stats, const std::string& column_name, const void* query_value, InvertedIndexQueryType query_type, - std::shared_ptr& r, + std::shared_ptr r, InvertedIndexVisitor* visitor); + Status handle_cache(InvertedIndexQueryCache* cache, + const InvertedIndexQueryCache::CacheKey& cache_key, + InvertedIndexQueryCacheHandle* cache_handler, OlapReaderStatistics* stats, + roaring::Roaring* bit_map); + InvertedIndexReaderType type() override; - Status get_bkd_reader(std::shared_ptr& reader); + Status get_bkd_reader(std::shared_ptr* reader); private: const TypeInfo* _type_info {}; From 9b494f4b3688bdcdc38c8b0c53b20ecc628dc4a4 Mon Sep 17 00:00:00 2001 From: bobhan1 Date: Thu, 7 Sep 2023 11:13:22 +0800 Subject: [PATCH 08/14] [Fix](autoinc) skip to fill the auto increment column when the input column is not nullable (#23905) --- be/src/vec/sink/vtablet_block_convertor.cpp | 12 ++-- .../unique/test_unique_table_auto_inc.out | 42 ++++++++++++ .../unique/test_unique_table_auto_inc.groovy | 67 +++++++++++++++++++ 3 files changed, 115 insertions(+), 6 deletions(-) diff --git a/be/src/vec/sink/vtablet_block_convertor.cpp b/be/src/vec/sink/vtablet_block_convertor.cpp index 45105e0cfffe4ef..92536697c9375ed 100644 --- a/be/src/vec/sink/vtablet_block_convertor.cpp +++ b/be/src/vec/sink/vtablet_block_convertor.cpp @@ -462,7 +462,6 @@ Status OlapTableBlockConvertor::_fill_auto_inc_cols(vectorized::Block* block, si vectorized::ColumnInt64::Container& dst_values = dst_column->get_data(); vectorized::ColumnPtr src_column_ptr = block->get_by_position(idx).column; - DCHECK(vectorized::is_column_const(*src_column_ptr) || src_column_ptr->is_nullable()); if (const vectorized::ColumnConst* const_column = check_and_get_column(src_column_ptr)) { // for insert stmt like "insert into tbl1 select null,col1,col2,... from tbl2" or @@ -487,11 +486,10 @@ Status OlapTableBlockConvertor::_fill_auto_inc_cols(vectorized::Block* block, si int64_t value = const_column->get_int(0); dst_values.resize_fill(rows, value); } - } else { - const auto& src_nullable_column = - assert_cast(*src_column_ptr); - auto src_nested_column_ptr = src_nullable_column.get_nested_column_ptr(); - const auto& null_map_data = src_nullable_column.get_null_map_data(); + } else if (const vectorized::ColumnNullable* src_nullable_column = + check_and_get_column(src_column_ptr)) { + auto src_nested_column_ptr = src_nullable_column->get_nested_column_ptr(); + const auto& null_map_data = src_nullable_column->get_null_map_data(); dst_values.reserve(rows); for (size_t i = 0; i < rows; i++) { null_value_count += null_map_data[i]; @@ -506,6 +504,8 @@ Status OlapTableBlockConvertor::_fill_auto_inc_cols(vectorized::Block* block, si dst_values.emplace_back((null_map_data[i] != 0) ? _auto_inc_id_allocator.next_id() : src_nested_column_ptr->get_int(i)); } + } else { + return Status::OK(); } block->get_by_position(idx).column = std::move(dst_column); block->get_by_position(idx).type = diff --git a/regression-test/data/data_model_p0/unique/test_unique_table_auto_inc.out b/regression-test/data/data_model_p0/unique/test_unique_table_auto_inc.out index 9245ebc6b167050..92328005edd20cc 100644 --- a/regression-test/data/data_model_p0/unique/test_unique_table_auto_inc.out +++ b/regression-test/data/data_model_p0/unique/test_unique_table_auto_inc.out @@ -153,3 +153,45 @@ Nereids 9998 8 1 b 20 2 c 30 +-- !sql -- +0 AFRICA lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to +1 AMERICA hs use ironic, even requests. s +2 ASIA ges. thinly even pinto beans ca +3 EUROPE ly final courts cajole furiously final excuse +4 MIDDLE EAST uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl + +-- !sql -- +0 AFRICA foobar +1 AMERICA foobar +2 ASIA foobar +3 EUROPE foobar +4 MIDDLE EAST foobar + +-- !sql -- +0 AFRICA barfoo +1 AMERICA barfoo +2 ASIA barfoo +3 EUROPE barfoo +4 MIDDLE EAST barfoo + +-- !sql -- +0 AFRICA lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to +1 AMERICA hs use ironic, even requests. s +2 ASIA ges. thinly even pinto beans ca +3 EUROPE ly final courts cajole furiously final excuse +4 MIDDLE EAST uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl + +-- !sql -- +0 AFRICA lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to +1 AMERICA hs use ironic, even requests. s +2 ASIA ges. thinly even pinto beans ca +3 test1 test2 +4 MIDDLE EAST uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl + +-- !sql -- +0 AFRICA lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to +1 AMERICA hs use ironic, even requests. s +2 ASIA ges. thinly even pinto beans ca +3 test1 test2 +4 test3 test4 + diff --git a/regression-test/suites/data_model_p0/unique/test_unique_table_auto_inc.groovy b/regression-test/suites/data_model_p0/unique/test_unique_table_auto_inc.groovy index 84fd02f6c25c061..0e5eac531f8aa35 100644 --- a/regression-test/suites/data_model_p0/unique/test_unique_table_auto_inc.groovy +++ b/regression-test/suites/data_model_p0/unique/test_unique_table_auto_inc.groovy @@ -342,5 +342,72 @@ suite("test_unique_table_auto_inc") { sql "drop table if exists ${table8};" sql "drop table if exists ${table9};" sql "drop table if exists ${table10};" + + + def table11 = "test_unique_tab_auto_inc_col_insert_select" + sql "drop table if exists ${table11}" + sql """CREATE TABLE ${table11} ( + `r_regionkey` bigint(20) NOT NULL AUTO_INCREMENT, + `r_name` varchar(25) NOT NULL, + `r_comment` varchar(152) NULL + ) ENGINE=OLAP + UNIQUE KEY(`r_regionkey`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`r_regionkey`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); """ + sql """ INSERT INTO ${table11} values + (0,'AFRICA','lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to') +,(1,'AMERICA','hs use ironic, even requests. s') +,(2,'ASIA','ges. thinly even pinto beans ca') +,(3,'EUROPE','ly final courts cajole furiously final excuse') +,(4,'MIDDLE EAST','uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl');""" + qt_sql "select * from ${table11} order by r_regionkey;" + sql 'set enable_nereids_planner=true' + sql "set experimental_enable_nereids_planner=true;" + sql 'set enable_nereids_dml=true' + sql "update ${table11} set r_comment = 'foobar' where r_regionkey <= 10;" + qt_sql "select * from ${table11} order by r_regionkey;" + + sql 'set enable_nereids_planner=false' + sql "set experimental_enable_nereids_planner=false;" + sql 'set enable_nereids_dml=false' + sql "update ${table11} set r_comment = 'barfoo' where r_regionkey <= 10;" + qt_sql "select * from ${table11} order by r_regionkey;" + sql "drop table if exists ${table11};" + + + def table12 = "test_unique_tab_auto_inc_col_insert_select2" + sql "drop table if exists ${table12}" + sql """CREATE TABLE ${table12} ( + `r_regionkey` bigint(20) NOT NULL AUTO_INCREMENT, + `r_name` varchar(25) NOT NULL, + `r_comment` varchar(152) NULL + ) ENGINE=OLAP + UNIQUE KEY(`r_regionkey`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`r_regionkey`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); """ + sql """ INSERT INTO ${table12} values + (0,'AFRICA','lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to') +,(1,'AMERICA','hs use ironic, even requests. s') +,(2,'ASIA','ges. thinly even pinto beans ca') +,(3,'EUROPE','ly final courts cajole furiously final excuse') +,(4,'MIDDLE EAST','uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl');""" + qt_sql "select * from ${table12} order by r_regionkey;" + sql 'set enable_nereids_planner=true' + sql "set experimental_enable_nereids_planner=true;" + sql 'set enable_nereids_dml=true' + sql """insert into ${table12} select r_regionkey, "test1", "test2" from ${table12} where r_regionkey=3;""" + qt_sql "select * from ${table12} order by r_regionkey;" + sql 'set enable_nereids_planner=false' + sql "set experimental_enable_nereids_planner=false;" + sql 'set enable_nereids_dml=false' + sql """insert into ${table12} select r_regionkey, "test3", "test4" from ${table12} where r_regionkey=4;""" + qt_sql "select * from ${table12} order by r_regionkey;" + sql "drop table if exists ${table12};" } From 349c090469e036b7ceaac87943149230b38da372 Mon Sep 17 00:00:00 2001 From: Kaijie Chen Date: Thu, 7 Sep 2023 11:19:21 +0800 Subject: [PATCH 09/14] [fix](move-memtable) lock when send data in load stream stub (#23949) --- be/src/vec/sink/load_stream_stub.cpp | 15 +++++++-------- be/src/vec/sink/load_stream_stub.h | 3 ++- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/be/src/vec/sink/load_stream_stub.cpp b/be/src/vec/sink/load_stream_stub.cpp index fbc69d3a9388454..63bea0b6c730f5c 100644 --- a/be/src/vec/sink/load_stream_stub.cpp +++ b/be/src/vec/sink/load_stream_stub.cpp @@ -215,16 +215,15 @@ Status LoadStreamStub::_encode_and_send(PStreamHeader& header, std::span lock(_buffer_mutex); - _buffer.append(buf); - if (eos || _buffer.size() >= config::brpc_streaming_client_batch_bytes) { - output.swap(_buffer); - } - } - if (output.size() == 0) { + std::unique_lock buffer_lock(_buffer_mutex); + _buffer.append(buf); + if (!eos && _buffer.size() < config::brpc_streaming_client_batch_bytes) { return Status::OK(); } + output.swap(_buffer); + // acquire send lock while holding buffer lock, to ensure the message order + std::lock_guard send_lock(_send_mutex); + buffer_lock.unlock(); VLOG_DEBUG << "send buf size : " << output.size() << ", eos: " << eos; return _send_with_retry(output); } diff --git a/be/src/vec/sink/load_stream_stub.h b/be/src/vec/sink/load_stream_stub.h index 2efa8875814accb..8a1ae79c52948f6 100644 --- a/be/src/vec/sink/load_stream_stub.h +++ b/be/src/vec/sink/load_stream_stub.h @@ -175,7 +175,8 @@ class LoadStreamStub { bthread::Mutex _mutex; bthread::ConditionVariable _close_cv; - bthread::Mutex _buffer_mutex; + std::mutex _buffer_mutex; + std::mutex _send_mutex; butil::IOBuf _buffer; PUniqueId _load_id; From f13d3008a262c9acf14ea1b0b0ef6097e8ec7464 Mon Sep 17 00:00:00 2001 From: Pxl Date: Thu, 7 Sep 2023 11:28:41 +0800 Subject: [PATCH 10/14] [Bug](type) coredump on get_type_as_primitive_type #24012 --- be/src/vec/data_types/data_type_nullable.cpp | 7 +++++-- be/src/vec/data_types/data_type_number_base.h | 5 ++++- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/be/src/vec/data_types/data_type_nullable.cpp b/be/src/vec/data_types/data_type_nullable.cpp index 1a1e302cad14e24..cfacd22b1edd53d 100644 --- a/be/src/vec/data_types/data_type_nullable.cpp +++ b/be/src/vec/data_types/data_type_nullable.cpp @@ -43,9 +43,12 @@ namespace doris::vectorized { DataTypeNullable::DataTypeNullable(const DataTypePtr& nested_data_type_) : nested_data_type {nested_data_type_} { + if (!nested_data_type) { + throw Exception(ErrorCode::INTERNAL_ERROR, "DataTypeNullable input nested type is nullptr"); + } if (!nested_data_type->can_be_inside_nullable()) { - LOG(FATAL) << fmt::format("Nested type {} cannot be inside Nullable type", - nested_data_type->get_name()); + throw Exception(ErrorCode::INTERNAL_ERROR, "Nested type {} cannot be inside Nullable type", + nested_data_type->get_name()); } } 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 2f8260c0bbd919f..6ed8047f34ef2fe 100644 --- a/be/src/vec/data_types/data_type_number_base.h +++ b/be/src/vec/data_types/data_type_number_base.h @@ -66,6 +66,9 @@ class DataTypeNumberBase : public IDataType { const char* get_family_name() const override { return TypeName::get(); } TypeIndex get_type_id() const override { return TypeId::value; } PrimitiveType get_type_as_primitive_type() const override { + if constexpr (std::is_same_v, TypeId>) { + return TYPE_BOOLEAN; + } if constexpr (std::is_same_v, TypeId>) { return TYPE_TINYINT; } @@ -87,7 +90,7 @@ class DataTypeNumberBase : public IDataType { if constexpr (std::is_same_v, TypeId>) { return TYPE_DOUBLE; } - __builtin_unreachable(); + return INVALID_TYPE; } TPrimitiveType::type get_type_as_tprimitive_type() const override { if constexpr (std::is_same_v, TypeId>) { From e40a4d4ae3a01f21575994d573bf3c1e6716ed00 Mon Sep 17 00:00:00 2001 From: DongLiang-0 <46414265+DongLiang-0@users.noreply.github.com> Date: Thu, 7 Sep 2023 12:55:52 +0800 Subject: [PATCH 11/14] [fix](auth)Fix create user no permissions of information_schema database (#23898) When creating a new user, this user does not have the information_schema database permission. --- .../org/apache/doris/mysql/privilege/Role.java | 17 ++++++++++++----- .../doris/mysql/privilege/RoleManager.java | 15 +++++++++------ .../suites/account_p0/test_alter_user.groovy | 10 ++++++++++ 3 files changed, 31 insertions(+), 11 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Role.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Role.java index 025b86fa638d621..7f1f8bb8efbbf42 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Role.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Role.java @@ -45,6 +45,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Objects; @@ -138,17 +139,23 @@ public Role(String roleName, WorkloadGroupPattern workloadGroupPattern, PrivBitS grantPrivs(workloadGroupPattern, privs.copy()); } - public Role(String roleName, TablePattern tablePattern, PrivBitSet tablePrivs, + public Role(String roleName, List tablePatterns, PrivBitSet tablePrivs, WorkloadGroupPattern workloadGroupPattern, PrivBitSet workloadGroupPrivs) { this.roleName = roleName; - this.tblPatternToPrivs.put(tablePattern, tablePrivs); this.workloadGroupPatternToPrivs.put(workloadGroupPattern, workloadGroupPrivs); - //for init admin role,will not generate exception + tablePatterns.forEach(tablePattern -> { + // for init admin role,will not generate exception + try { + this.tblPatternToPrivs.put(tablePattern, tablePrivs); + grantPrivs(tablePattern, tablePrivs.copy()); + } catch (DdlException e) { + LOG.warn("grant table failed,", e); + } + }); try { - grantPrivs(tablePattern, tablePrivs.copy()); grantPrivs(workloadGroupPattern, workloadGroupPrivs.copy()); } catch (DdlException e) { - LOG.warn("grant failed,", e); + LOG.warn("grant workload group failed,", e); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/RoleManager.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/RoleManager.java index 7b37b6f40e99980..7df0baf495b9fd7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/RoleManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/RoleManager.java @@ -192,17 +192,20 @@ public Role createDefaultRole(UserIdentity userIdent) throws DdlException { if (roles.containsKey(userDefaultRoleName)) { return roles.get(userDefaultRoleName); } + // grant read privs to database information_schema & mysql - TablePattern tblPattern = new TablePattern(Auth.DEFAULT_CATALOG, InfoSchemaDb.DATABASE_NAME, "*"); + List tablePatterns = Lists.newArrayList(); + TablePattern informationTblPattern = new TablePattern(Auth.DEFAULT_CATALOG, InfoSchemaDb.DATABASE_NAME, "*"); try { - tblPattern.analyze(SystemInfoService.DEFAULT_CLUSTER); + informationTblPattern.analyze(SystemInfoService.DEFAULT_CLUSTER); + tablePatterns.add(informationTblPattern); } catch (AnalysisException e) { LOG.warn("should not happen", e); } - - tblPattern = new TablePattern(Auth.DEFAULT_CATALOG, MysqlDb.DATABASE_NAME, "*"); + TablePattern mysqlTblPattern = new TablePattern(Auth.DEFAULT_CATALOG, MysqlDb.DATABASE_NAME, "*"); try { - tblPattern.analyze(SystemInfoService.DEFAULT_CLUSTER); + mysqlTblPattern.analyze(SystemInfoService.DEFAULT_CLUSTER); + tablePatterns.add(mysqlTblPattern); } catch (AnalysisException e) { LOG.warn("should not happen", e); } @@ -214,7 +217,7 @@ public Role createDefaultRole(UserIdentity userIdent) throws DdlException { } catch (AnalysisException e) { LOG.warn("should not happen", e); } - Role role = new Role(userDefaultRoleName, tblPattern, PrivBitSet.of(Privilege.SELECT_PRIV), + Role role = new Role(userDefaultRoleName, tablePatterns, PrivBitSet.of(Privilege.SELECT_PRIV), workloadGroupPattern, PrivBitSet.of(Privilege.USAGE_PRIV)); roles.put(role.getRoleName(), role); return role; diff --git a/regression-test/suites/account_p0/test_alter_user.groovy b/regression-test/suites/account_p0/test_alter_user.groovy index 445e70109282735..9414025ef0490c3 100644 --- a/regression-test/suites/account_p0/test_alter_user.groovy +++ b/regression-test/suites/account_p0/test_alter_user.groovy @@ -192,5 +192,15 @@ suite("test_alter_user", "account") { result2 = connect(user = 'test_auth_user4', password = '12345', url = context.config.jdbcUrl) { sql 'select 1' } + + // 9. test user default database privileges + sql """drop user if exists test_auth_user4""" + sql """create user test_auth_user4 identified by '12345'""" + sql """grant SELECT_PRIV on regression_test.* to test_auth_user4""" + result1 = connect(user = 'test_auth_user4', password = '12345', url = context.config.jdbcUrl) { + sql 'select 1' + sql 'use information_schema' + sql 'use mysql' + } } From 7adab1138cb8a50d678ac9536e53deb8a0e823c5 Mon Sep 17 00:00:00 2001 From: Kang Date: Thu, 7 Sep 2023 14:48:13 +0800 Subject: [PATCH 12/14] [improvement](index) support CANCEL BUILD INDEX (#23760) Currently, BUILD INDEX can not be canceled. This pr add CANCEL BUILD INDEX as follows: `CANCEL BUILD INDEX index_name ON table_name [(jobid1,jobid2)]` --- .../docs/data-table/index/inverted-index.md | 40 ++-- .../docs/data-table/index/inverted-index.md | 34 ++-- fe/fe-core/src/main/cup/sql_parser.cup | 10 +- .../apache/doris/alter/IndexChangeJob.java | 37 +++- .../org/apache/doris/alter/RollupJobV2.java | 2 +- .../doris/alter/SchemaChangeHandler.java | 61 +++++++ .../apache/doris/analysis/ShowAlterStmt.java | 2 +- .../org/apache/doris/catalog/DatabaseIf.java | 9 + .../java/org/apache/doris/catalog/Env.java | 3 +- .../inverted_index_p0/test_build_index.groovy | 172 ++++++++++++++++++ 10 files changed, 331 insertions(+), 39 deletions(-) create mode 100644 regression-test/suites/inverted_index_p0/test_build_index.groovy diff --git a/docs/en/docs/data-table/index/inverted-index.md b/docs/en/docs/data-table/index/inverted-index.md index c711b1fca47222c..1e17ca011b46f16 100644 --- a/docs/en/docs/data-table/index/inverted-index.md +++ b/docs/en/docs/data-table/index/inverted-index.md @@ -119,22 +119,28 @@ ALTER TABLE table_name ADD INDEX idx_name(column_name) USING INVERTED [PROPERTIE **After version 2.0-beta (including 2.0-beta):** -The above 'create/add index' operation only generates inverted index for incremental data. The syntax of build index is added to add inverted index to stock data: +The above 'create/add index' operation only generates inverted index for incremental data. The syntax of BUILD INDEX is added to add inverted index to stock data: ```sql -- syntax 1, add inverted index to the stock data of the whole table by default BUILD INDEX index_name ON table_name; -- syntax 2, partition can be specified, and one or more can be specified BUILD INDEX index_name ON table_name PARTITIONS(partition_name1, partition_name2); ``` -(**The above 'create/add index' operation needs to be executed before executing the build index**) +(**The above 'create/add index' operation needs to be executed before executing the BUILD INDEX**) -To view the progress of the `build index`, you can use the following statement +To view the progress of the `BUILD INDEX`, you can run the following statement ```sql -show build index [FROM db_name]; --- Example 1: Viewing the progress of all build index tasks -show build index; --- Example 2: Viewing the progress of the build index task for a specified table -show build index where TableName = "table1"; +SHOW BUILD INDEX [FROM db_name]; +-- Example 1: Viewing the progress of all BUILD INDEX tasks +SHOW BUILD INDEX; +-- Example 2: Viewing the progress of the BUILD INDEX task for a specified table +SHOW BUILD INDEX where TableName = "table1"; +``` + +To cancel `BUILD INDEX`, you can run the following statement +```sql +CANCEL BUILD INDEX ON table_name; +CANCEL BUILD INDEX ON table_name (job_id1,jobid_2,...); ``` - drop an inverted index @@ -403,13 +409,13 @@ mysql> SELECT count() FROM hackernews_1m WHERE timestamp > '2007-08-23 04:17:00' mysql> CREATE INDEX idx_timestamp ON hackernews_1m(timestamp) USING INVERTED; Query OK, 0 rows affected (0.03 sec) ``` -**After 2.0-beta (including 2.0-beta), you need to execute `build index` to add inverted index to the stock data:** +**After 2.0-beta (including 2.0-beta), you need to execute `BUILD INDEX` to add inverted index to the stock data:** ```sql mysql> BUILD INDEX idx_timestamp ON hackernews_1m; Query OK, 0 rows affected (0.01 sec) ``` -- progress of building index can be view by SQL. It just costs 1s (compare FinishTime and CreateTime) to build index for timestamp column with 1 million rows. +- progress of building index can be view by SQL. It just costs 1s (compare FinishTime and CreateTime) to BUILD INDEX for timestamp column with 1 million rows. ```sql mysql> SHOW ALTER TABLE COLUMN; +-------+---------------+-------------------------+-------------------------+---------------+---------+---------------+---------------+---------------+----------+------+----------+---------+ @@ -420,10 +426,10 @@ mysql> SHOW ALTER TABLE COLUMN; 1 row in set (0.00 sec) ``` -**After 2.0-beta (including 2.0-beta), you can view the progress of stock data creating index by `show build index`:** +**After 2.0-beta (including 2.0-beta), you can view the progress of stock data creating index by `SHOW BUILD INDEX`:** ```sql -- If the table has no partitions, the PartitionName defaults to TableName -mysql> show build index; +mysql> SHOW BUILD INDEX; +-------+---------------+---------------+----------------------------------------------------------+-------------------------+-------------------------+---------------+----------+------+----------+ | JobId | TableName | PartitionName | AlterInvertedIndexes | CreateTime | FinishTime | TransactionId | State | Msg | Progress | +-------+---------------+---------------+----------------------------------------------------------+-------------------------+-------------------------+---------------+----------+------+----------+ @@ -458,7 +464,7 @@ mysql> SELECT count() FROM hackernews_1m WHERE parent = 11189; mysql> ALTER TABLE hackernews_1m ADD INDEX idx_parent(parent) USING INVERTED; Query OK, 0 rows affected (0.01 sec) --- After 2.0-beta (including 2.0-beta), you need to execute `build index` to add inverted index to the stock data: +-- After 2.0-beta (including 2.0-beta), you need to execute `BUILD INDEX` to add inverted index to the stock data: mysql> BUILD INDEX idx_parent ON hackernews_1m; Query OK, 0 rows affected (0.01 sec) @@ -470,7 +476,7 @@ mysql> SHOW ALTER TABLE COLUMN; | 10053 | hackernews_1m | 2023-02-10 19:49:32.893 | 2023-02-10 19:49:33.982 | hackernews_1m | 10054 | 10008 | 1:378856428 | 4 | FINISHED | | NULL | 2592000 | +-------+---------------+-------------------------+-------------------------+---------------+---------+---------------+---------------+---------------+----------+------+----------+---------+ -mysql> show build index; +mysql> SHOW BUILD INDEX; +-------+---------------+---------------+----------------------------------------------------+-------------------------+-------------------------+---------------+----------+------+----------+ | JobId | TableName | PartitionName | AlterInvertedIndexes | CreateTime | FinishTime | TransactionId | State | Msg | Progress | +-------+---------------+---------------+----------------------------------------------------+-------------------------+-------------------------+---------------+----------+------+----------+ @@ -501,11 +507,11 @@ mysql> SELECT count() FROM hackernews_1m WHERE author = 'faster'; mysql> ALTER TABLE hackernews_1m ADD INDEX idx_author(author) USING INVERTED; Query OK, 0 rows affected (0.01 sec) --- After 2.0-beta (including 2.0-beta), you need to execute `build index` to add inverted index to the stock data: +-- After 2.0-beta (including 2.0-beta), you need to execute `BUILD INDEX` to add inverted index to the stock data: mysql> BUILD INDEX idx_author ON hackernews_1m; Query OK, 0 rows affected (0.01 sec) --- costs 1.5s to build index for author column with 1 million rows. +-- costs 1.5s to BUILD INDEX for author column with 1 million rows. mysql> SHOW ALTER TABLE COLUMN; +-------+---------------+-------------------------+-------------------------+---------------+---------+---------------+---------------+---------------+----------+------+----------+---------+ | JobId | TableName | CreateTime | FinishTime | IndexName | IndexId | OriginIndexId | SchemaVersion | TransactionId | State | Msg | Progress | Timeout | @@ -515,7 +521,7 @@ mysql> SHOW ALTER TABLE COLUMN; | 10076 | hackernews_1m | 2023-02-10 19:54:20.046 | 2023-02-10 19:54:21.521 | hackernews_1m | 10077 | 10008 | 1:1335127701 | 5 | FINISHED | | NULL | 2592000 | +-------+---------------+-------------------------+-------------------------+---------------+---------+---------------+---------------+---------------+----------+------+----------+---------+ -mysql> show build index order by CreateTime desc limit 1; +mysql> SHOW BUILD INDEX order by CreateTime desc limit 1; +-------+---------------+---------------+----------------------------------------------------+-------------------------+-------------------------+---------------+----------+------+----------+ | JobId | TableName | PartitionName | AlterInvertedIndexes | CreateTime | FinishTime | TransactionId | State | Msg | Progress | +-------+---------------+---------------+----------------------------------------------------+-------------------------+-------------------------+---------------+----------+------+----------+ diff --git a/docs/zh-CN/docs/data-table/index/inverted-index.md b/docs/zh-CN/docs/data-table/index/inverted-index.md index 01ee5d1ea923b0d..ce859737520d734 100644 --- a/docs/zh-CN/docs/data-table/index/inverted-index.md +++ b/docs/zh-CN/docs/data-table/index/inverted-index.md @@ -117,22 +117,28 @@ ALTER TABLE table_name ADD INDEX idx_name(column_name) USING INVERTED [PROPERTIE **2.0-beta版本(含2.0-beta)之后:** -上述`create/add index`操作只对增量数据生成倒排索引,增加了build index的语法用于对存量数据加倒排索引: +上述`create/add index`操作只对增量数据生成倒排索引,增加了BUILD INDEX的语法用于对存量数据加倒排索引: ```sql -- 语法1,默认给全表的存量数据加上倒排索引 BUILD INDEX index_name ON table_name; -- 语法2,可指定partition,可指定一个或多个 BUILD INDEX index_name ON table_name PARTITIONS(partition_name1, partition_name2); ``` -(**在执行build index之前需要已经执行了以上`create/add index`的操作**) +(**在执行BUILD INDEX之前需要已经执行了以上`create/add index`的操作**) -查看`build index`进展,可通过以下语句进行查看: +查看`BUILD INDEX`进展,可通过以下语句进行查看: ```sql -show build index [FROM db_name]; --- 示例1,查看所有的build index任务进展 -show build index; --- 示例2,查看指定table的build index任务进展 -show build index where TableName = "table1"; +SHOW BUILD INDEX [FROM db_name]; +-- 示例1,查看所有的BUILD INDEX任务进展 +SHOW BUILD INDEX; +-- 示例2,查看指定table的BUILD INDEX任务进展 +SHOW BUILD INDEX where TableName = "table1"; +``` + +取消 `BUILD INDEX`, 可通过以下语句进行 +```sql +CANCEL BUILD INDEX ON table_name; +CANCEL BUILD INDEX ON table_name (job_id1,jobid_2,...); ``` - 删除倒排索引 @@ -401,7 +407,7 @@ mysql> SELECT count() FROM hackernews_1m WHERE timestamp > '2007-08-23 04:17:00' mysql> CREATE INDEX idx_timestamp ON hackernews_1m(timestamp) USING INVERTED; Query OK, 0 rows affected (0.03 sec) ``` - **2.0-beta(含2.0-beta)后,需要再执行`build index`才能给存量数据加上倒排索引:** + **2.0-beta(含2.0-beta)后,需要再执行`BUILD INDEX`才能给存量数据加上倒排索引:** ```sql mysql> BUILD INDEX idx_timestamp ON hackernews_1m; Query OK, 0 rows affected (0.01 sec) @@ -421,7 +427,7 @@ mysql> SHOW ALTER TABLE COLUMN; **2.0-beta(含2.0-beta)后,可通过`show builde index`来查看存量数据创建索引进展:** ```sql -- 若table没有分区,PartitionName默认就是TableName -mysql> show build index; +mysql> SHOW BUILD INDEX; +-------+---------------+---------------+----------------------------------------------------------+-------------------------+-------------------------+---------------+----------+------+----------+ | JobId | TableName | PartitionName | AlterInvertedIndexes | CreateTime | FinishTime | TransactionId | State | Msg | Progress | +-------+---------------+---------------+----------------------------------------------------------+-------------------------+-------------------------+---------------+----------+------+----------+ @@ -456,7 +462,7 @@ mysql> SELECT count() FROM hackernews_1m WHERE parent = 11189; mysql> ALTER TABLE hackernews_1m ADD INDEX idx_parent(parent) USING INVERTED; Query OK, 0 rows affected (0.01 sec) --- 2.0-beta(含2.0-beta)后,需要再执行build index才能给存量数据加上倒排索引: +-- 2.0-beta(含2.0-beta)后,需要再执行BUILD INDEX才能给存量数据加上倒排索引: mysql> BUILD INDEX idx_parent ON hackernews_1m; Query OK, 0 rows affected (0.01 sec) @@ -468,7 +474,7 @@ mysql> SHOW ALTER TABLE COLUMN; | 10053 | hackernews_1m | 2023-02-10 19:49:32.893 | 2023-02-10 19:49:33.982 | hackernews_1m | 10054 | 10008 | 1:378856428 | 4 | FINISHED | | NULL | 2592000 | +-------+---------------+-------------------------+-------------------------+---------------+---------+---------------+---------------+---------------+----------+------+----------+---------+ -mysql> show build index; +mysql> SHOW BUILD INDEX; +-------+---------------+---------------+----------------------------------------------------+-------------------------+-------------------------+---------------+----------+------+----------+ | JobId | TableName | PartitionName | AlterInvertedIndexes | CreateTime | FinishTime | TransactionId | State | Msg | Progress | +-------+---------------+---------------+----------------------------------------------------+-------------------------+-------------------------+---------------+----------+------+----------+ @@ -499,7 +505,7 @@ mysql> SELECT count() FROM hackernews_1m WHERE author = 'faster'; mysql> ALTER TABLE hackernews_1m ADD INDEX idx_author(author) USING INVERTED; Query OK, 0 rows affected (0.01 sec) --- 2.0-beta(含2.0-beta)后,需要再执行build index才能给存量数据加上倒排索引: +-- 2.0-beta(含2.0-beta)后,需要再执行BUILD INDEX才能给存量数据加上倒排索引: mysql> BUILD INDEX idx_author ON hackernews_1m; Query OK, 0 rows affected (0.01 sec) @@ -513,7 +519,7 @@ mysql> SHOW ALTER TABLE COLUMN; | 10076 | hackernews_1m | 2023-02-10 19:54:20.046 | 2023-02-10 19:54:21.521 | hackernews_1m | 10077 | 10008 | 1:1335127701 | 5 | FINISHED | | NULL | 2592000 | +-------+---------------+-------------------------+-------------------------+---------------+---------+---------------+---------------+---------------+----------+------+----------+---------+ -mysql> show build index order by CreateTime desc limit 1; +mysql> SHOW BUILD INDEX order by CreateTime desc limit 1; +-------+---------------+---------------+----------------------------------------------------+-------------------------+-------------------------+---------------+----------+------+----------+ | JobId | TableName | PartitionName | AlterInvertedIndexes | CreateTime | FinishTime | TransactionId | State | Msg | Progress | +-------+---------------+---------------+----------------------------------------------------+-------------------------+-------------------------+---------------+----------+------+----------+ diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index 2aa1a993d842ab7..3cd7f9fd3fe6c20 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -855,7 +855,7 @@ nonterminal MultiPartitionDesc multi_partition_desc; nonterminal List privilege_list; nonterminal List string_list; -nonterminal List integer_list, cancel_rollup_job_id_list; +nonterminal List integer_list, job_id_list; nonterminal AccessPrivilegeWithCols privilege_type; nonterminal DataDescription data_desc, mysql_data_desc; @@ -4505,7 +4505,7 @@ cancel_stmt ::= :} ; -cancel_rollup_job_id_list ::= +job_id_list ::= {: RESULT = null; :} @@ -4524,10 +4524,14 @@ cancel_param ::= {: RESULT = new CancelExportStmt(db, parser.where); :} - | KW_ALTER KW_TABLE opt_alter_type:type KW_FROM table_name:table cancel_rollup_job_id_list:list + | KW_ALTER KW_TABLE opt_alter_type:type KW_FROM table_name:table job_id_list:list {: RESULT = new CancelAlterTableStmt(type, table, list); :} + | KW_BUILD KW_INDEX KW_ON table_name:table job_id_list:list + {: + RESULT = new CancelAlterTableStmt(ShowAlterStmt.AlterType.INDEX, table, list); + :} | KW_DECOMMISSION KW_BACKEND string_list:hostPorts {: RESULT = new CancelAlterSystemStmt(hostPorts); 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 6ba3e68ffa582da..e6f4c4e0a0e4e58 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 @@ -41,6 +41,7 @@ import org.apache.doris.task.AgentTaskQueue; import org.apache.doris.task.AlterInvertedIndexTask; import org.apache.doris.thrift.TColumn; +import org.apache.doris.thrift.TTaskType; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; @@ -321,10 +322,31 @@ protected void runRunningJob() throws AlterCancelException { LOG.info("inverted index job finished: {}", jobId); } + /** + * cancelImpl() can be called any time any place. + * We need to clean any possible residual of this job. + */ protected boolean cancelImpl(String errMsg) { + if (jobState.isFinalState()) { + return false; + } + + cancelInternal(); + + jobState = JobState.CANCELLED; + this.errMsg = errMsg; + this.finishedTimeMs = System.currentTimeMillis(); + Env.getCurrentEnv().getEditLog().logIndexChangeJob(this); + LOG.info("cancel index job {}, err: {}", jobId, errMsg); return true; } + private void cancelInternal() { + // clear tasks if has + AgentTaskQueue.removeBatchTask(invertedIndexBatchTask, TTaskType.ALTER_INVERTED_INDEX); + // TODO maybe delete already build index files + } + public void replay(IndexChangeJob replayedJob) { try { IndexChangeJob replayedIndexChangeJob = (IndexChangeJob) replayedJob; @@ -336,8 +358,7 @@ public void replay(IndexChangeJob replayedJob) { replayRunningJob(replayedIndexChangeJob); break; case CANCELLED: - // TODO: - // replayCancelled(replayedIndexChangeJob); + replayCancelled(replayedIndexChangeJob); break; default: break; @@ -361,6 +382,18 @@ private void replayRunningJob(IndexChangeJob replayedJob) { LOG.info("replay finished inverted index job: {} table id: {}", jobId, tableId); } + /** + * Replay job in CANCELLED state. + */ + private void replayCancelled(IndexChangeJob replayedJob) { + cancelInternal(); + + this.jobState = JobState.CANCELLED; + this.errMsg = replayedJob.errMsg; + this.finishedTimeMs = replayedJob.finishedTimeMs; + LOG.info("cancel index job {}, err: {}", jobId, errMsg); + } + 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/alter/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java index f4f274e6d243a73..8d4af3648bbd69d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java @@ -639,8 +639,8 @@ protected boolean cancelImpl(String errMsg) { jobState = JobState.CANCELLED; this.errMsg = errMsg; this.finishedTimeMs = System.currentTimeMillis(); - LOG.info("cancel {} job {}, err: {}", this.type, jobId, errMsg); Env.getCurrentEnv().getEditLog().logAlterJob(this); + LOG.info("cancel {} job {}, err: {}", this.type, jobId, errMsg); return true; } 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 8bd5fab817c1336..3de1d43163a2216 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 @@ -32,6 +32,7 @@ import org.apache.doris.analysis.ModifyColumnClause; import org.apache.doris.analysis.ModifyTablePropertiesClause; import org.apache.doris.analysis.ReorderColumnsClause; +import org.apache.doris.analysis.ShowAlterStmt.AlterType; import org.apache.doris.catalog.AggregateType; import org.apache.doris.catalog.BinlogConfig; import org.apache.doris.catalog.Column; @@ -2336,7 +2337,14 @@ public void updatePartitionProperties(Database db, String tableName, String part @Override public void cancel(CancelStmt stmt) throws DdlException { CancelAlterTableStmt cancelAlterTableStmt = (CancelAlterTableStmt) stmt; + if (cancelAlterTableStmt.getAlterType() == AlterType.INDEX) { + cancelIndexJob(cancelAlterTableStmt); + } else { + cancelColumnJob(cancelAlterTableStmt); + } + } + private void cancelColumnJob(CancelAlterTableStmt cancelAlterTableStmt) throws DdlException { String dbName = cancelAlterTableStmt.getDbName(); String tableName = cancelAlterTableStmt.getTableName(); Preconditions.checkState(!Strings.isNullOrEmpty(dbName)); @@ -2375,6 +2383,59 @@ public void cancel(CancelStmt stmt) throws DdlException { } } + private void cancelIndexJob(CancelAlterTableStmt cancelAlterTableStmt) throws DdlException { + String dbName = cancelAlterTableStmt.getDbName(); + String tableName = cancelAlterTableStmt.getTableName(); + Preconditions.checkState(!Strings.isNullOrEmpty(dbName)); + Preconditions.checkState(!Strings.isNullOrEmpty(tableName)); + + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(dbName); + + List jobList = new ArrayList<>(); + + Table olapTable = db.getTableOrDdlException(tableName, Table.TableType.OLAP); + olapTable.writeLock(); + try { + // find from index change jobs first + if (cancelAlterTableStmt.getAlterJobIdList() != null + && cancelAlterTableStmt.getAlterJobIdList().size() > 0) { + for (Long jobId : cancelAlterTableStmt.getAlterJobIdList()) { + IndexChangeJob job = indexChangeJobs.get(jobId); + if (job == null) { + continue; + } + jobList.add(job); + LOG.debug("add build index job {} on table {} for specific id", jobId, tableName); + } + } else { + for (IndexChangeJob job : indexChangeJobs.values()) { + if (!job.isDone() && job.getTableId() == olapTable.getId()) { + jobList.add(job); + LOG.debug("add build index job {} on table {} for all", job.getJobId(), tableName); + } + } + } + } finally { + olapTable.writeUnlock(); + } + + // alter job v2's cancel must be called outside the table lock + if (jobList.size() > 0) { + for (IndexChangeJob job : jobList) { + long jobId = job.getJobId(); + LOG.debug("cancel build index job {} on table {}", jobId, tableName); + if (!job.cancel("user cancelled")) { + LOG.warn("cancel build index job {} on table {} failed", jobId, tableName); + throw new DdlException("Job can not be cancelled. State: " + job.getJobState()); + } else { + LOG.info("cancel build index job {} on table {} success", jobId, tableName); + } + } + } else { + throw new DdlException("No job to cancel for Table[" + tableName + "]"); + } + } + /** * Returns true if the index already exists, there is no need to create the job to add the index. * Otherwise return false, there is need to create a job to add the index. diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java index 0b7017c49b28353..625d04d2dc839c3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java @@ -53,7 +53,7 @@ public class ShowAlterStmt extends ShowStmt { private static final Logger LOG = LogManager.getLogger(ShowAlterStmt.class); public static enum AlterType { - COLUMN, ROLLUP + COLUMN, ROLLUP, INDEX } private AlterType type; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java index 8c2833764123fa1..7dfceabbdea4f79 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java @@ -202,6 +202,15 @@ default T getTableOrDdlException(String tableName) throws DdlException { return getTableOrException(tableName, t -> new DdlException(ErrorCode.ERR_BAD_TABLE_ERROR.formatErrorMsg(t))); } + default T getTableOrDdlException(String tableName, TableIf.TableType tableType) throws DdlException { + T table = getTableOrDdlException(tableName); + if (table.getType() != tableType) { + throw new DdlException( + "table type is not " + tableType + ", tableName=" + tableName + ", type=" + table.getType()); + } + return table; + } + default T getTableOrDdlException(long tableId) throws DdlException { return getTableOrException(tableId, t -> new DdlException(ErrorCode.ERR_BAD_TABLE_ERROR.formatErrorMsg(t))); } 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 4066b5877592fa2..873894e7dfaf95d 100755 --- 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 @@ -4033,7 +4033,8 @@ public void refreshMaterializedView(RefreshMaterializedViewStmt stmt) throws Ddl public void cancelAlter(CancelAlterTableStmt stmt) throws DdlException { if (stmt.getAlterType() == AlterType.ROLLUP) { this.getMaterializedViewHandler().cancel(stmt); - } else if (stmt.getAlterType() == AlterType.COLUMN) { + } else if (stmt.getAlterType() == AlterType.COLUMN + || stmt.getAlterType() == AlterType.INDEX) { this.getSchemaChangeHandler().cancel(stmt); } else { throw new DdlException("Cancel " + stmt.getAlterType() + " does not implement yet"); diff --git a/regression-test/suites/inverted_index_p0/test_build_index.groovy b/regression-test/suites/inverted_index_p0/test_build_index.groovy new file mode 100644 index 000000000000000..13bf046592339f8 --- /dev/null +++ b/regression-test/suites/inverted_index_p0/test_build_index.groovy @@ -0,0 +1,172 @@ +// 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_build_index", "inverted_index"){ + // prepare test table + def timeout = 60000 + def delta_time = 1000 + def alter_res = "null" + def useTime = 0 + + def wait_for_latest_op_on_table_finish = { table_name, OpTimeout -> + for(int t = delta_time; t <= OpTimeout; t += delta_time){ + alter_res = sql """SHOW ALTER TABLE COLUMN WHERE TableName = "${table_name}" ORDER BY CreateTime DESC LIMIT 1;""" + alter_res = alter_res.toString() + if(alter_res.contains("FINISHED")) { + sleep(3000) // wait change table state to normal + logger.info(table_name + " latest alter job finished, detail: " + alter_res) + break + } + useTime = t + sleep(delta_time) + } + assertTrue(useTime <= OpTimeout, "wait_for_latest_op_on_table_finish timeout") + } + + def wait_for_build_index_on_partition_finish = { table_name, OpTimeout -> + for(int t = delta_time; t <= OpTimeout; t += delta_time){ + alter_res = sql """SHOW BUILD INDEX WHERE TableName = "${table_name}";""" + def expected_finished_num = alter_res.size(); + def finished_num = 0; + for (int i = 0; i < expected_finished_num; i++) { + logger.info(table_name + " build index job state: " + alter_res[i][7] + i) + if (alter_res[i][7] == "FINISHED") { + ++finished_num; + } + } + if (finished_num == expected_finished_num) { + logger.info(table_name + " all build index jobs finished, detail: " + alter_res) + break + } + useTime = t + sleep(delta_time) + } + assertTrue(useTime <= OpTimeout, "wait_for_latest_build_index_on_partition_finish timeout") + } + + def wait_for_last_build_index_on_table_finish = { table_name, OpTimeout -> + for(int t = delta_time; t <= OpTimeout; t += delta_time){ + alter_res = sql """SHOW BUILD INDEX WHERE TableName = "${table_name}" ORDER BY JobId """ + + def last_job_state = alter_res[alter_res.size()-1][7]; + if (last_job_state == "FINISHED" || last_job_state == "CANCELLED") { + logger.info(table_name + " last index job finished, state: " + last_job_state + ", detail: " + alter_res) + return last_job_state; + } + useTime = t + sleep(delta_time) + } + assertTrue(useTime <= OpTimeout, "wait_for_last_build_index_on_table_finish timeout") + return "wait_timeout" + } + + def tableName = "hackernews_1m" + + sql "DROP TABLE IF EXISTS ${tableName}" + // create 1 replica table + sql """ + CREATE TABLE ${tableName} ( + `id` bigint(20) NULL, + `deleted` tinyint(4) NULL, + `type` text NULL, + `author` text NULL, + `timestamp` datetime NULL, + `comment` text NULL, + `dead` tinyint(4) NULL, + `parent` bigint(20) NULL, + `poll` bigint(20) NULL, + `children` array NULL, + `url` text NULL, + `score` int(11) NULL, + `title` text NULL, + `parts` array NULL, + `descendants` int(11) NULL + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "is_being_synced" = "false", + "storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false" + ); + """ + + // stream load data + streamLoad { + table "${tableName}" + + set 'compress_type', 'GZ' + + file """${getS3Url()}/regression/index/hacknernews_1m.csv.gz""" + + time 60000 // limit inflight 60s + + // stream load action will check result, include Success status, and NumberTotalRows == NumberLoadedRows + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + 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" + + sql """ ALTER TABLE ${tableName} ADD INDEX idx_comment (`comment`) USING INVERTED PROPERTIES("parser" = "english") """ + + sql """ BUILD INDEX idx_comment ON ${tableName} """ + + sleep(1000) + + def result = sql """ SHOW BUILD INDEX WHERE TableName = "${tableName}" ORDER BY JobId """ + assertEquals(result[result.size()-1][1], tableName) + assertTrue(result[result.size()-1][3].contains("ADD INDEX")) + assertEquals(result[result.size()-1][7], "RUNNING") + + sql """ CANCEL BUILD INDEX ON ${tableName} (${result[result.size()-1][0]}) """ + result = sql """ SHOW BUILD INDEX WHERE TableName = "${tableName}" ORDER BY JobId """ + assertEquals(result[result.size()-1][1], tableName) + assertTrue(result[result.size()-1][3].contains("ADD INDEX")) + assertEquals(result[result.size()-1][7], "CANCELLED") + assertEquals(result[result.size()-1][8], "user cancelled") + + + sql """ BUILD INDEX idx_comment ON ${tableName}; """ + def state = wait_for_last_build_index_on_table_finish(tableName, timeout) + assertEquals(state, "FINISHED") + + def success = false; + try { + sql """ CANCEL BUILD INDEX ON ${tableName}; """ + success = true + } catch(Exception ex) { + logger.info(" CANCEL BUILD INDEX ON ${tableName} exception: " + ex) + } + assertFalse(success) +} From 4a455f6a4c4c102f01bb63d860bf31ef992abd97 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=A2=E5=81=A5?= Date: Thu, 7 Sep 2023 15:00:40 +0800 Subject: [PATCH 13/14] [feature](Nereids) multi array contains to array overlap (#23864) transform ``` array_contains ( c_array, '1' ) OR array_contains ( c_array, '2' ) ``` to ``` array_overlap(c_array, ['1', '2']) ``` --- .../expression/ExpressionOptimization.java | 4 +- .../rules/ArrayContainToArrayOverlap.java | 99 +++++++++++++++++++ .../ArrayContainsToArrayOverlapTest.java | 99 +++++++++++++++++++ 3 files changed, 201 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ArrayContainToArrayOverlap.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ArrayContainsToArrayOverlapTest.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java index 53f360d5ffe6c37..3f30d10ebe587c0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.rules.expression; +import org.apache.doris.nereids.rules.expression.rules.ArrayContainToArrayOverlap; import org.apache.doris.nereids.rules.expression.rules.DistinctPredicatesRule; import org.apache.doris.nereids.rules.expression.rules.ExtractCommonFactorRule; import org.apache.doris.nereids.rules.expression.rules.OrToIn; @@ -40,7 +41,8 @@ public class ExpressionOptimization extends ExpressionRewrite { SimplifyInPredicate.INSTANCE, SimplifyDecimalV3Comparison.INSTANCE, SimplifyRange.INSTANCE, - OrToIn.INSTANCE + OrToIn.INSTANCE, + ArrayContainToArrayOverlap.INSTANCE ); private static final ExpressionRuleExecutor EXECUTOR = new ExpressionRuleExecutor(OPTIMIZE_REWRITE_RULES); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ArrayContainToArrayOverlap.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ArrayContainToArrayOverlap.java new file mode 100644 index 000000000000000..7309ef111c925d0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ArrayContainToArrayOverlap.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.rules.expression.rules; + +import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; +import org.apache.doris.nereids.rules.expression.ExpressionRewriteRule; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Or; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayContains; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ArraysOverlap; +import org.apache.doris.nereids.trees.expressions.literal.ArrayLiteral; +import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter; +import org.apache.doris.nereids.util.ExpressionUtils; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableList.Builder; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * array_contains ( c_array, '1' ) + * OR array_contains ( c_array, '2' ) + * =========================================> + * array_overlap(c_array, ['1', '2']) + */ +public class ArrayContainToArrayOverlap extends DefaultExpressionRewriter implements + ExpressionRewriteRule { + + public static final ArrayContainToArrayOverlap INSTANCE = new ArrayContainToArrayOverlap(); + + private static final int REWRITE_PREDICATE_THRESHOLD = 2; + + @Override + public Expression rewrite(Expression expr, ExpressionRewriteContext ctx) { + return expr.accept(this, ctx); + } + + @Override + public Expression visitOr(Or or, ExpressionRewriteContext ctx) { + List disjuncts = ExpressionUtils.extractDisjunction(or); + Map> containFuncAndOtherFunc = disjuncts.stream() + .collect(Collectors.partitioningBy(this::isValidArrayContains)); + Map> containLiteralSet = new HashMap<>(); + List contains = containFuncAndOtherFunc.get(true); + List others = containFuncAndOtherFunc.get(false); + + contains.forEach(containFunc -> + containLiteralSet.computeIfAbsent(containFunc.child(0), k -> new HashSet<>()) + .add((Literal) containFunc.child(1))); + + Builder newDisjunctsBuilder = new ImmutableList.Builder<>(); + containLiteralSet.forEach((left, literalSet) -> { + if (literalSet.size() > REWRITE_PREDICATE_THRESHOLD) { + newDisjunctsBuilder.add( + new ArraysOverlap(left, + new ArrayLiteral(ImmutableList.copyOf(literalSet)))); + } + }); + + contains.stream() + .filter(e -> !canCovertToArrayOverlap(e, containLiteralSet)) + .forEach(newDisjunctsBuilder::add); + others.stream() + .map(e -> e.accept(this, null)) + .forEach(newDisjunctsBuilder::add); + return ExpressionUtils.or(newDisjunctsBuilder.build()); + } + + private boolean isValidArrayContains(Expression expression) { + return expression instanceof ArrayContains && expression.child(1) instanceof Literal; + } + + private boolean canCovertToArrayOverlap(Expression expression, Map> containLiteralSet) { + return expression instanceof ArrayContains + && containLiteralSet.getOrDefault(expression.child(0), + new HashSet<>()).size() > REWRITE_PREDICATE_THRESHOLD; + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ArrayContainsToArrayOverlapTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ArrayContainsToArrayOverlapTest.java new file mode 100644 index 000000000000000..dfee1a7caed4b7a --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ArrayContainsToArrayOverlapTest.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.rules.rewrite; + +import org.apache.doris.nereids.rules.expression.ExpressionRewriteTestHelper; +import org.apache.doris.nereids.trees.expressions.And; +import org.apache.doris.nereids.trees.expressions.EqualTo; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Or; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ArraysOverlap; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.util.MemoTestUtils; +import org.apache.doris.nereids.util.PlanChecker; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +class ArrayContainsToArrayOverlapTest extends ExpressionRewriteTestHelper { + + @Test + void testOr() { + String sql = "select array_contains([1], 1) or array_contains([1], 2) or array_contains([1], 3);"; + Plan plan = PlanChecker.from(MemoTestUtils.createConnectContext()) + .analyze(sql) + .rewrite() + .getPlan(); + Expression expression = plan.child(0).getExpressions().get(0).child(0); + Assertions.assertTrue(expression instanceof ArraysOverlap); + Assertions.assertEquals("array(1)", expression.child(0).toSql()); + Assertions.assertEquals("array(1, 2, 3)", expression.child(1).toSql()); + } + + @Test + void testAnd() { + String sql = "select array_contains([1], 1) " + + "or array_contains([1], 2) " + + "or array_contains([1], 3)" + + "or array_contains([1], 4) and array_contains([1], 5);"; + Plan plan = PlanChecker.from(MemoTestUtils.createConnectContext()) + .analyze(sql) + .rewrite() + .getPlan(); + Expression expression = plan.child(0).getExpressions().get(0).child(0); + Assertions.assertTrue(expression instanceof Or); + Assertions.assertTrue(expression.child(0) instanceof ArraysOverlap); + Assertions.assertTrue(expression.child(1) instanceof And); + } + + @Test + void testAndOther() { + String sql = "select bin(0) == 1 " + + "or array_contains([1], 1) " + + "or array_contains([1], 2) " + + "or array_contains([1], 3) " + + "or array_contains([1], 4) and array_contains([1], 5);"; + Plan plan = PlanChecker.from(MemoTestUtils.createConnectContext()) + .analyze(sql) + .rewrite() + .getPlan(); + Expression expression = plan.child(0).getExpressions().get(0).child(0); + Assertions.assertTrue(expression instanceof Or); + Assertions.assertTrue(expression.child(0) instanceof Or); + Assertions.assertTrue(expression.child(0).child(0) instanceof ArraysOverlap); + Assertions.assertTrue(expression.child(0).child(1) instanceof EqualTo); + Assertions.assertTrue(expression.child(1) instanceof And); + } + + @Test + void testAndOverlap() { + String sql = "select array_contains([1], 0) " + + "or (array_contains([1], 1) " + + "and (array_contains([1], 2) " + + "or array_contains([1], 3) " + + "or array_contains([1], 4)));"; + Plan plan = PlanChecker.from(MemoTestUtils.createConnectContext()) + .analyze(sql) + .rewrite() + .getPlan(); + Expression expression = plan.child(0).getExpressions().get(0).child(0); + Assertions.assertEquals("(array_contains(array(1), 0) OR " + + "(array_contains(array(1), 1) AND arrays_overlap(array(1), array(2, 3, 4))))", + expression.toSql()); + } +} From fdb7a44f57b074f35d3b7457562015bcf754ba85 Mon Sep 17 00:00:00 2001 From: zclllyybb Date: Thu, 7 Sep 2023 17:08:26 +0800 Subject: [PATCH 14/14] Revert "[Feature](partitions) Support auto partition" (#24024) * Revert "[Feature](partitions) Support auto partition (#23236)" This reverts commit 6c544dd2011d731b8c9c51384c77bcf19c017981. * Update config.h --- be/src/common/config.h | 2 +- be/src/common/status.h | 3 - be/src/exec/tablet_info.cpp | 282 ++----- be/src/exec/tablet_info.h | 90 +-- be/src/exprs/runtime_filter.cpp | 84 ++- be/src/exprs/runtime_filter.h | 140 +++- be/src/pipeline/pipeline_task.cpp | 4 - be/src/runtime/load_channel.cpp | 7 +- be/src/runtime/plan_fragment_executor.cpp | 9 +- be/src/runtime/tablets_channel.cpp | 90 +-- be/src/runtime/tablets_channel.h | 4 +- be/src/vec/core/block.cpp | 8 +- be/src/vec/core/block.h | 2 +- be/src/vec/exprs/vexpr.cpp | 87 --- be/src/vec/exprs/vexpr.h | 142 ---- be/src/vec/exprs/vliteral.cpp | 1 - be/src/vec/sink/vtablet_finder.cpp | 44 +- be/src/vec/sink/vtablet_finder.h | 2 +- be/src/vec/sink/vtablet_sink.cpp | 706 ++++++------------ be/src/vec/sink/vtablet_sink.h | 61 +- .../apache/doris/common/FeMetaVersion.java | 4 +- fe/fe-core/src/main/cup/sql_parser.cup | 23 +- .../doris/analysis/ListPartitionDesc.java | 21 +- .../apache/doris/analysis/PartitionDesc.java | 81 +- .../doris/analysis/PartitionExprUtil.java | 193 ----- .../doris/analysis/RangePartitionDesc.java | 19 +- .../doris/catalog/ListPartitionInfo.java | 41 +- .../org/apache/doris/catalog/OlapTable.java | 2 +- .../apache/doris/catalog/PartitionInfo.java | 36 +- .../doris/catalog/RangePartitionInfo.java | 41 +- .../apache/doris/planner/OlapTableSink.java | 31 +- .../doris/service/FrontendServiceImpl.java | 135 ---- .../doris/catalog/TruncateTableTest.java | 13 +- .../common/util/DynamicPartitionUtilTest.java | 2 +- .../service/FrontendServiceImplTest.java | 159 ---- gensrc/proto/internal_service.proto | 1 - gensrc/thrift/Descriptors.thrift | 4 - gensrc/thrift/FrontendService.thrift | 17 - .../auto_partition_stream_load1.csv | 10 - .../auto_partition_stream_load2.csv | 10 - .../test_auto_list_partition.out | 41 - .../test_auto_partition_load.out | 25 - .../test_auto_range_partition.out | 74 -- .../test_auto_list_partition.groovy | 91 --- .../test_auto_partition_load.groovy | 79 -- .../test_auto_range_partition.groovy | 90 --- .../partition_p0/test_datev2_partition.groovy | 2 + 47 files changed, 602 insertions(+), 2411 deletions(-) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionExprUtil.java delete mode 100644 fe/fe-core/src/test/java/org/apache/doris/service/FrontendServiceImplTest.java delete mode 100644 regression-test/data/partition_p0/auto_partition/auto_partition_stream_load1.csv delete mode 100644 regression-test/data/partition_p0/auto_partition/auto_partition_stream_load2.csv delete mode 100644 regression-test/data/partition_p0/auto_partition/test_auto_list_partition.out delete mode 100644 regression-test/data/partition_p0/auto_partition/test_auto_partition_load.out delete mode 100644 regression-test/data/partition_p0/auto_partition/test_auto_range_partition.out delete mode 100644 regression-test/suites/partition_p0/auto_partition/test_auto_list_partition.groovy delete mode 100644 regression-test/suites/partition_p0/auto_partition/test_auto_partition_load.groovy delete mode 100644 regression-test/suites/partition_p0/auto_partition/test_auto_range_partition.groovy diff --git a/be/src/common/config.h b/be/src/common/config.h index 78ee4f9fefe40a3..61c9cc7d08a42aa 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -168,7 +168,7 @@ DECLARE_mString(process_full_gc_size); // used memory and the exec_mem_limit will be canceled. // If false, cancel query when the memory used exceeds exec_mem_limit, same as before. DECLARE_mBool(enable_query_memory_overcommit); - +//waibibabu // gc will release cache, cancel task, and task will wait for gc to release memory, // default gc strategy is conservative, if you want to exclude the interference of gc, let it be true DECLARE_mBool(disable_memory_gc); diff --git a/be/src/common/status.h b/be/src/common/status.h index 44c840a2d096f50..14b91dd7d002ce3 100644 --- a/be/src/common/status.h +++ b/be/src/common/status.h @@ -113,7 +113,6 @@ E(NOT_INITIALIZED, -236); E(ALREADY_CANCELLED, -237); E(TOO_MANY_SEGMENTS, -238); E(ALREADY_CLOSED, -239); -E(NEED_SEND_AGAIN, -240); E(CE_CMD_PARAMS_ERROR, -300); E(CE_BUFFER_TOO_SMALL, -301); E(CE_CMD_NOT_VALID, -302); @@ -285,7 +284,6 @@ constexpr bool capture_stacktrace(int code) { && code != ErrorCode::TOO_MANY_VERSION && code != ErrorCode::ALREADY_CANCELLED && code != ErrorCode::ALREADY_CLOSED - && code != ErrorCode::NEED_SEND_AGAIN && code != ErrorCode::PUSH_TRANSACTION_ALREADY_EXIST && code != ErrorCode::BE_NO_SUITABLE_VERSION && code != ErrorCode::CUMULATIVE_NO_SUITABLE_VERSION @@ -429,7 +427,6 @@ class Status { ERROR_CTOR(DataQualityError, DATA_QUALITY_ERROR) ERROR_CTOR(NotAuthorized, NOT_AUTHORIZED) ERROR_CTOR(HttpError, HTTP_ERROR) - ERROR_CTOR(NeedSendAgain, NEED_SEND_AGAIN) #undef ERROR_CTOR template diff --git a/be/src/exec/tablet_info.cpp b/be/src/exec/tablet_info.cpp index 71ca504d3a010ab..764dddf4e9b9fd2 100644 --- a/be/src/exec/tablet_info.cpp +++ b/be/src/exec/tablet_info.cpp @@ -27,10 +27,7 @@ #include #include -#include -#include "common/exception.h" -#include "common/status.h" #include "olap/tablet_schema.h" #include "runtime/descriptors.h" #include "runtime/large_int_value.h" @@ -40,11 +37,8 @@ #include "util/hash_util.hpp" #include "util/string_parser.hpp" #include "util/string_util.h" -#include "vec/columns/column.h" -#include "vec/columns/column_nullable.h" -#include "vec/common/assert_cast.h" #include "vec/common/string_ref.h" -#include "vec/exprs/vliteral.h" +#include "vec/exprs/vexpr.h" #include "vec/runtime/vdatetime_value.h" namespace doris { @@ -63,61 +57,6 @@ void OlapTableIndexSchema::to_protobuf(POlapTableIndexSchema* pindex) const { } } -bool VOlapTablePartKeyComparator::operator()(const BlockRowWithIndicator lhs, - const BlockRowWithIndicator rhs) const { - vectorized::Block* l_block = std::get<0>(lhs); - vectorized::Block* r_block = std::get<0>(rhs); - int32_t l_row = std::get<1>(lhs); - int32_t r_row = std::get<1>(rhs); - bool l_use_new = std::get<2>(lhs); - bool r_use_new = std::get<2>(rhs); - - if (l_row == -1) { - return false; - } else if (r_row == -1) { - return true; - } - - if (_param_locs.empty()) { // no transform, use origin column - for (auto slot_loc : _slot_locs) { - auto res = l_block->get_by_position(slot_loc).column->compare_at( - l_row, r_row, *r_block->get_by_position(slot_loc).column, -1); - if (res != 0) { - return res < 0; - } - } - } else { // use transformed column to compare - DCHECK(_slot_locs.size() == _param_locs.size()) - << _slot_locs.size() << ' ' << _param_locs.size(); - - //TODO: use template to accelerate this for older compiler. - const std::vector* l_index = l_use_new ? &_param_locs : &_slot_locs; - const std::vector* r_index = r_use_new ? &_param_locs : &_slot_locs; - - for (int i = 0; i < _slot_locs.size(); i++) { - vectorized::ColumnPtr l_col = l_block->get_by_position((*l_index)[i]).column; - vectorized::ColumnPtr r_col = r_block->get_by_position((*r_index)[i]).column; - //TODO: when we support any function for transform, maybe the best way is refactor all doris' functions to its essential nullable mode. - if (auto* nullable = - vectorized::check_and_get_column(l_col)) { - l_col = nullable->get_nested_column_ptr(); - } - if (auto* nullable = - vectorized::check_and_get_column(r_col)) { - r_col = nullable->get_nested_column_ptr(); - } - - auto res = l_col->compare_at(l_row, r_row, *r_col, -1); - if (res != 0) { - return res < 0; - } - } - } - - // equal, return false - return false; -} - Status OlapTableSchemaParam::init(const POlapTableSchemaParam& pschema) { _db_id = pschema.db_id(); _table_id = pschema.table_id(); @@ -269,23 +208,11 @@ VOlapTablePartitionParam::VOlapTablePartitionParam(std::shared_ptrtuple_desc()->slots()), - _mem_tracker(std::make_unique("OlapTablePartitionParam")), - _part_type(t_param.partition_type) { + _mem_tracker(std::make_unique("OlapTablePartitionParam")) { for (auto slot : _slots) { _partition_block.insert( {slot->get_empty_mutable_column(), slot->get_data_type_ptr(), slot->col_name()}); } - - if (t_param.__isset.enable_automatic_partition && t_param.enable_automatic_partition) { - _is_auto_partiton = true; - Status st = vectorized::VExpr::create_expr_tree(t_param.partition_function_exprs[0], - _part_func_ctx); - if (!st.ok()) { - throw Exception(Status::InternalError("Partition function expr is not valid"), - "Partition function expr is not valid"); - } - _partition_function = _part_func_ctx->root(); - } } VOlapTablePartitionParam::~VOlapTablePartitionParam() { @@ -316,8 +243,8 @@ Status VOlapTablePartitionParam::init() { } _partitions_map.reset( - new std::map( - VOlapTablePartKeyComparator(_partition_slot_locs, _transformed_slot_locs))); + new std::map( + VOlapTablePartKeyComparator(_partition_slot_locs))); if (_t_param.__isset.distributed_columns) { for (auto& col : _t_param.distributed_columns) { RETURN_IF_ERROR(find_slot_locs(col, _distributed_slot_locs, "distributed")); @@ -345,22 +272,67 @@ Status VOlapTablePartitionParam::init() { }; } - // for both auto/non-auto partition table. - _is_in_partition = _part_type == TPartitionType::type::LIST_PARTITIONED; + DCHECK(!_t_param.partitions.empty()) << "must have at least 1 partition"; + _is_in_partition = _t_param.partitions[0].__isset.in_keys; // initial partitions for (int i = 0; i < _t_param.partitions.size(); ++i) { const TOlapTablePartition& t_part = _t_param.partitions[i]; - VOlapTablePartition* part = nullptr; - RETURN_IF_ERROR(generate_partition_from(t_part, part)); + auto part = _obj_pool.add(new VOlapTablePartition(&_partition_block)); + part->id = t_part.id; + part->is_mutable = t_part.is_mutable; + + if (!_is_in_partition) { + if (t_part.__isset.start_keys) { + RETURN_IF_ERROR(_create_partition_keys(t_part.start_keys, &part->start_key)); + } + + if (t_part.__isset.end_keys) { + RETURN_IF_ERROR(_create_partition_keys(t_part.end_keys, &part->end_key)); + } + } else { + for (const auto& keys : t_part.in_keys) { + RETURN_IF_ERROR(_create_partition_keys( + keys, &part->in_keys.emplace_back(&_partition_block, -1))); + } + if (t_part.__isset.is_default_partition && t_part.is_default_partition) { + _default_partition = part; + } + } + + part->num_buckets = t_part.num_buckets; + auto num_indexes = _schema->indexes().size(); + if (t_part.indexes.size() != num_indexes) { + return Status::InternalError( + "number of partition's index is not equal with schema's" + ", num_part_indexes={}, num_schema_indexes={}", + t_part.indexes.size(), num_indexes); + } + part->indexes = t_part.indexes; + std::sort(part->indexes.begin(), part->indexes.end(), + [](const OlapTableIndexTablets& lhs, const OlapTableIndexTablets& rhs) { + return lhs.index_id < rhs.index_id; + }); + // check index + for (int j = 0; j < num_indexes; ++j) { + if (part->indexes[j].index_id != _schema->indexes()[j]->index_id) { + std::stringstream ss; + ss << "partition's index is not equal with schema's" + << ", part_index=" << part->indexes[j].index_id + << ", schema_index=" << _schema->indexes()[j]->index_id; + return Status::InternalError( + "partition's index is not equal with schema's" + ", part_index={}, schema_index={}", + part->indexes[j].index_id, _schema->indexes()[j]->index_id); + } + } _partitions.emplace_back(part); if (_is_in_partition) { for (auto& in_key : part->in_keys) { - _partitions_map->emplace(std::tuple {in_key.first, in_key.second, false}, part); + _partitions_map->emplace(&in_key, part); } } else { - _partitions_map->emplace(std::tuple {part->end_key.first, part->end_key.second, false}, - part); + _partitions_map->emplace(&part->end_key, part); } } @@ -371,32 +343,19 @@ Status VOlapTablePartitionParam::init() { bool VOlapTablePartitionParam::find_partition(BlockRow* block_row, const VOlapTablePartition** partition) const { - // block_row is gave by inserting process. So try to use transformed index. - auto it = - _is_in_partition - ? _partitions_map->find(std::tuple {block_row->first, block_row->second, true}) - : _partitions_map->upper_bound( - std::tuple {block_row->first, block_row->second, true}); + auto it = _is_in_partition ? _partitions_map->find(block_row) + : _partitions_map->upper_bound(block_row); // for list partition it might result in default partition if (_is_in_partition) { *partition = (it != _partitions_map->end()) ? it->second : _default_partition; it = _partitions_map->end(); } - if (it != _partitions_map->end() && - _part_contains(it->second, std::tuple {block_row->first, block_row->second, true})) { + if (it != _partitions_map->end() && _part_contains(it->second, block_row)) { *partition = it->second; } return (*partition != nullptr); } -bool VOlapTablePartitionParam::_part_contains(VOlapTablePartition* part, - BlockRowWithIndicator key) const { - // start_key.second == -1 means only single partition - VOlapTablePartKeyComparator comparator(_partition_slot_locs, _transformed_slot_locs); - return part->start_key.second == -1 || - !comparator(key, std::tuple {part->start_key.first, part->start_key.second, false}); -} - uint32_t VOlapTablePartitionParam::find_tablet(BlockRow* block_row, const VOlapTablePartition& partition) const { return _compute_tablet_index(block_row, partition.num_buckets); @@ -410,61 +369,6 @@ Status VOlapTablePartitionParam::_create_partition_keys(const std::vectorid = t_part.id; - part_result->is_mutable = t_part.is_mutable; - - if (!_is_in_partition) { - if (t_part.__isset.start_keys) { - RETURN_IF_ERROR(_create_partition_keys(t_part.start_keys, &part_result->start_key)); - } - - if (t_part.__isset.end_keys) { - RETURN_IF_ERROR(_create_partition_keys(t_part.end_keys, &part_result->end_key)); - } - } else { - for (const auto& keys : t_part.in_keys) { - RETURN_IF_ERROR(_create_partition_keys( - keys, &part_result->in_keys.emplace_back(&_partition_block, -1))); - } - if (t_part.__isset.is_default_partition && t_part.is_default_partition && - _default_partition == nullptr) { - _default_partition = part_result; - } - } - - part_result->num_buckets = t_part.num_buckets; - auto num_indexes = _schema->indexes().size(); - if (t_part.indexes.size() != num_indexes) { - return Status::InternalError( - "number of partition's index is not equal with schema's" - ", num_part_indexes={}, num_schema_indexes={}", - t_part.indexes.size(), num_indexes); - } - part_result->indexes = t_part.indexes; - std::sort(part_result->indexes.begin(), part_result->indexes.end(), - [](const OlapTableIndexTablets& lhs, const OlapTableIndexTablets& rhs) { - return lhs.index_id < rhs.index_id; - }); - // check index - for (int j = 0; j < num_indexes; ++j) { - if (part_result->indexes[j].index_id != _schema->indexes()[j]->index_id) { - std::stringstream ss; - ss << "partition's index is not equal with schema's" - << ", part_index=" << part_result->indexes[j].index_id - << ", schema_index=" << _schema->indexes()[j]->index_id; - return Status::InternalError( - "partition's index is not equal with schema's" - ", part_index={}, schema_index={}", - part_result->indexes[j].index_id, _schema->indexes()[j]->index_id); - } - } - return Status::OK(); -} - Status VOlapTablePartitionParam::_create_partition_key(const TExprNode& t_expr, BlockRow* part_key, uint16_t pos) { auto column = std::move(*part_key->first->get_by_position(pos).column).mutate(); @@ -553,72 +457,4 @@ Status VOlapTablePartitionParam::_create_partition_key(const TExprNode& t_expr, return Status::OK(); } -Status VOlapTablePartitionParam::add_partitions( - const std::vector& partitions) { - for (const auto& t_part : partitions) { - auto part = _obj_pool.add(new VOlapTablePartition(&_partition_block)); - part->id = t_part.id; - part->is_mutable = t_part.is_mutable; - - DCHECK(t_part.__isset.start_keys == t_part.__isset.end_keys && - t_part.__isset.start_keys != t_part.__isset.in_keys); - // range partition - if (t_part.__isset.start_keys) { - RETURN_IF_ERROR(_create_partition_keys(t_part.start_keys, &part->start_key)); - } - if (t_part.__isset.end_keys) { - RETURN_IF_ERROR(_create_partition_keys(t_part.end_keys, &part->end_key)); - } - // list partition - we only set 1 value in 1 partition for new created ones - if (t_part.__isset.in_keys) { - for (const auto& keys : t_part.in_keys) { - RETURN_IF_ERROR(_create_partition_keys( - keys, &part->in_keys.emplace_back(&_partition_block, -1))); - } - if (t_part.__isset.is_default_partition && t_part.is_default_partition) { - _default_partition = part; - } - } - - part->num_buckets = t_part.num_buckets; - auto num_indexes = _schema->indexes().size(); - if (t_part.indexes.size() != num_indexes) { - return Status::InternalError( - "number of partition's index is not equal with schema's" - ", num_part_indexes={}, num_schema_indexes={}", - t_part.indexes.size(), num_indexes); - } - part->indexes = t_part.indexes; - std::sort(part->indexes.begin(), part->indexes.end(), - [](const OlapTableIndexTablets& lhs, const OlapTableIndexTablets& rhs) { - return lhs.index_id < rhs.index_id; - }); - // check index - for (int j = 0; j < num_indexes; ++j) { - if (part->indexes[j].index_id != _schema->indexes()[j]->index_id) { - std::stringstream ss; - ss << "partition's index is not equal with schema's" - << ", part_index=" << part->indexes[j].index_id - << ", schema_index=" << _schema->indexes()[j]->index_id; - return Status::InternalError( - "partition's index is not equal with schema's" - ", part_index={}, schema_index={}", - part->indexes[j].index_id, _schema->indexes()[j]->index_id); - } - } - _partitions.emplace_back(part); - // after _creating_partiton_keys - if (_is_in_partition) { - for (auto& in_key : part->in_keys) { - _partitions_map->emplace(std::tuple {in_key.first, in_key.second, false}, part); - } - } else { - _partitions_map->emplace(std::tuple {part->end_key.first, part->end_key.second, false}, - part); - } - } - - return Status::OK(); -} - } // namespace doris diff --git a/be/src/exec/tablet_info.h b/be/src/exec/tablet_info.h index 3e6ab7b94be9224..c508b322ee172f5 100644 --- a/be/src/exec/tablet_info.h +++ b/be/src/exec/tablet_info.h @@ -26,7 +26,6 @@ #include #include #include -#include #include #include #include @@ -36,8 +35,6 @@ #include "vec/columns/column.h" #include "vec/core/block.h" #include "vec/core/column_with_type_and_name.h" -#include "vec/exprs/vexpr.h" -#include "vec/exprs/vexpr_context.h" #include "vec/exprs/vexpr_fwd.h" namespace doris { @@ -113,8 +110,7 @@ using OlapTableIndexTablets = TOlapTableIndexTablets; // } using BlockRow = std::pair; -using BlockRowWithIndicator = - std::tuple; // [block, column, is_transformed] +using VecBlock = vectorized::Block; struct VOlapTablePartition { int64_t id = 0; @@ -129,20 +125,32 @@ struct VOlapTablePartition { : start_key {partition_block, -1}, end_key {partition_block, -1} {} }; -// this is only used by tablet_sink. so we can assume it's inited by its' descriptor. class VOlapTablePartKeyComparator { public: - VOlapTablePartKeyComparator(const std::vector& slot_locs, - const std::vector& params_locs) - : _slot_locs(slot_locs), _param_locs(params_locs) {} + VOlapTablePartKeyComparator(const std::vector& slot_locs) : _slot_locs(slot_locs) {} // return true if lhs < rhs - // 'row' is -1 mean maximal boundary - bool operator()(const BlockRowWithIndicator lhs, const BlockRowWithIndicator rhs) const; + // 'row' is -1 mean + bool operator()(const BlockRow* lhs, const BlockRow* rhs) const { + if (lhs->second == -1) { + return false; + } else if (rhs->second == -1) { + return true; + } + + for (auto slot_loc : _slot_locs) { + auto res = lhs->first->get_by_position(slot_loc).column->compare_at( + lhs->second, rhs->second, *rhs->first->get_by_position(slot_loc).column, -1); + if (res != 0) { + return res < 0; + } + } + // equal, return false + return false; + } private: const std::vector& _slot_locs; - const std::vector& _param_locs; }; // store an olap table's tablet information @@ -166,26 +174,6 @@ class VOlapTablePartitionParam { const std::vector& get_partitions() const { return _partitions; } - // it's same with auto now because we only support transformed partition in auto partition. may expand in future - bool is_projection_partition() const { return _is_auto_partiton; } - bool is_auto_partition() const { return _is_auto_partiton; } - - std::vector get_partition_keys() const { return _partition_slot_locs; } - - Status add_partitions(const std::vector& partitions); - - //TODO: use vector when we support multi partition column for auto-partition - vectorized::VExprContextSPtr get_part_func_ctx() { return _part_func_ctx; } - vectorized::VExprSPtr get_partition_function() { return _partition_function; } - - // which will affect _partition_block - Status generate_partition_from(const TOlapTablePartition& t_part, - VOlapTablePartition*& part_result); - - void set_transformed_slots(const std::vector& new_slots) { - _transformed_slot_locs = new_slots; - } - private: Status _create_partition_keys(const std::vector& t_exprs, BlockRow* part_key); @@ -194,7 +182,11 @@ class VOlapTablePartitionParam { std::function _compute_tablet_index; // check if this partition contain this key - bool _part_contains(VOlapTablePartition* part, BlockRowWithIndicator key) const; + bool _part_contains(VOlapTablePartition* part, BlockRow* key) const { + // start_key.second == -1 means only single partition + VOlapTablePartKeyComparator comparator(_partition_slot_locs); + return part->start_key.second == -1 || !comparator(key, &part->start_key); + } // this partition only valid in this schema std::shared_ptr _schema; @@ -202,32 +194,21 @@ class VOlapTablePartitionParam { const std::vector& _slots; std::vector _partition_slot_locs; - std::vector _transformed_slot_locs; std::vector _distributed_slot_locs; ObjectPool _obj_pool; vectorized::Block _partition_block; std::unique_ptr _mem_tracker; std::vector _partitions; - // For all partition value rows saved in this map, indicator is false. whenever we use a value to find in it, the param is true. - // so that we can distinguish which column index to use (origin slots or transformed slots). - std::unique_ptr< - std::map> + std::unique_ptr> _partitions_map; bool _is_in_partition = false; uint32_t _mem_usage = 0; // only works when using list partition, the resource is owned by _partitions VOlapTablePartition* _default_partition = nullptr; - - // for auto partition, now only support 1 column. TODO: use vector to save them when we support multi column auto-partition. - bool _is_auto_partiton = false; - vectorized::VExprContextSPtr _part_func_ctx = nullptr; - vectorized::VExprSPtr _partition_function = nullptr; - TPartitionType::type _part_type; // support list or range }; -// indicate where's the tablet and all its replications (node-wise) using TabletLocation = TTabletLocation; // struct TTabletLocation { // 1: required i64 tablet_id @@ -254,17 +235,9 @@ class OlapTableLocationParam { return nullptr; } - void add_locations(std::vector& locations) { - for (auto& location : locations) { - if (_tablets.find(location.tablet_id) == _tablets.end()) { - _tablets[location.tablet_id] = &location; - } - } - } - private: TOlapTableLocationParam _t_param; - // [tablet_id, tablet]. tablet has id, also. + std::unordered_map _tablets; }; @@ -305,15 +278,6 @@ class DorisNodesInfo { return nullptr; } - void add_nodes(const std::vector& t_nodes) { - for (const auto& node : t_nodes) { - auto node_info = find_node(node.id); - if (node_info == nullptr) { - _nodes.emplace(node.id, node); - } - } - } - const std::unordered_map& nodes_info() { return _nodes; } private: diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index 4fd0e7b779b169a..3df28bfd37b72b7 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -193,7 +193,89 @@ PFilterType get_type(RuntimeFilterType type) { } Status create_literal(const TypeDescriptor& type, const void* data, vectorized::VExprSPtr& expr) { - TExprNode node = create_texpr_node_from(data, type.type, type.precision, type.scale); + TExprNode node; + + switch (type.type) { + case TYPE_BOOLEAN: { + create_texpr_literal_node(data, &node); + break; + } + case TYPE_TINYINT: { + create_texpr_literal_node(data, &node); + break; + } + case TYPE_SMALLINT: { + create_texpr_literal_node(data, &node); + break; + } + case TYPE_INT: { + create_texpr_literal_node(data, &node); + break; + } + case TYPE_BIGINT: { + create_texpr_literal_node(data, &node); + break; + } + case TYPE_LARGEINT: { + create_texpr_literal_node(data, &node); + break; + } + case TYPE_FLOAT: { + create_texpr_literal_node(data, &node); + break; + } + case TYPE_DOUBLE: { + create_texpr_literal_node(data, &node); + break; + } + case TYPE_DATEV2: { + create_texpr_literal_node(data, &node); + break; + } + case TYPE_DATETIMEV2: { + create_texpr_literal_node(data, &node); + break; + } + case TYPE_DATE: { + create_texpr_literal_node(data, &node); + break; + } + case TYPE_DATETIME: { + create_texpr_literal_node(data, &node); + break; + } + case TYPE_DECIMALV2: { + create_texpr_literal_node(data, &node, type.precision, type.scale); + break; + } + case TYPE_DECIMAL32: { + create_texpr_literal_node(data, &node, type.precision, type.scale); + break; + } + case TYPE_DECIMAL64: { + create_texpr_literal_node(data, &node, type.precision, type.scale); + break; + } + case TYPE_DECIMAL128I: { + create_texpr_literal_node(data, &node, type.precision, type.scale); + break; + } + case TYPE_CHAR: { + create_texpr_literal_node(data, &node); + break; + } + case TYPE_VARCHAR: { + create_texpr_literal_node(data, &node); + break; + } + case TYPE_STRING: { + create_texpr_literal_node(data, &node); + break; + } + default: + DCHECK(false); + return Status::InvalidArgument("Invalid type!"); + } try { expr = vectorized::VLiteral::create_shared(node); diff --git a/be/src/exprs/runtime_filter.h b/be/src/exprs/runtime_filter.h index 8fa603e4a12f978..3bd3efd6cc8cf95 100644 --- a/be/src/exprs/runtime_filter.h +++ b/be/src/exprs/runtime_filter.h @@ -42,7 +42,6 @@ #include "vec/common/string_ref.h" #include "vec/core/types.h" #include "vec/data_types/data_type.h" -#include "vec/exprs/vexpr.h" #include "vec/runtime/vdatetime_value.h" namespace butil { @@ -426,4 +425,143 @@ class RuntimeFilterWrapperHolder { private: WrapperPtr _wrapper; }; + +// copied from expr.h since it is only used in runtime filter + +template +Status create_texpr_literal_node(const void* data, TExprNode* node, int precision = 0, + int scale = 0) { + if constexpr (T == TYPE_BOOLEAN) { + auto origin_value = reinterpret_cast(data); + TBoolLiteral boolLiteral; + (*node).__set_node_type(TExprNodeType::BOOL_LITERAL); + boolLiteral.__set_value(*origin_value); + (*node).__set_bool_literal(boolLiteral); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_BOOLEAN)); + } else if constexpr (T == TYPE_TINYINT) { + auto origin_value = reinterpret_cast(data); + (*node).__set_node_type(TExprNodeType::INT_LITERAL); + TIntLiteral intLiteral; + intLiteral.__set_value(*origin_value); + (*node).__set_int_literal(intLiteral); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_TINYINT)); + } else if constexpr (T == TYPE_SMALLINT) { + auto origin_value = reinterpret_cast(data); + (*node).__set_node_type(TExprNodeType::INT_LITERAL); + TIntLiteral intLiteral; + intLiteral.__set_value(*origin_value); + (*node).__set_int_literal(intLiteral); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_SMALLINT)); + } else if constexpr (T == TYPE_INT) { + auto origin_value = reinterpret_cast(data); + (*node).__set_node_type(TExprNodeType::INT_LITERAL); + TIntLiteral intLiteral; + intLiteral.__set_value(*origin_value); + (*node).__set_int_literal(intLiteral); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_INT)); + } else if constexpr (T == TYPE_BIGINT) { + auto origin_value = reinterpret_cast(data); + (*node).__set_node_type(TExprNodeType::INT_LITERAL); + TIntLiteral intLiteral; + intLiteral.__set_value(*origin_value); + (*node).__set_int_literal(intLiteral); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_BIGINT)); + } else if constexpr (T == TYPE_LARGEINT) { + auto origin_value = reinterpret_cast(data); + (*node).__set_node_type(TExprNodeType::LARGE_INT_LITERAL); + TLargeIntLiteral large_int_literal; + large_int_literal.__set_value(LargeIntValue::to_string(*origin_value)); + (*node).__set_large_int_literal(large_int_literal); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_LARGEINT)); + } else if constexpr ((T == TYPE_DATE) || (T == TYPE_DATETIME) || (T == TYPE_TIME)) { + auto origin_value = reinterpret_cast(data); + TDateLiteral date_literal; + char convert_buffer[30]; + origin_value->to_string(convert_buffer); + date_literal.__set_value(convert_buffer); + (*node).__set_date_literal(date_literal); + (*node).__set_node_type(TExprNodeType::DATE_LITERAL); + if (origin_value->type() == TimeType::TIME_DATE) { + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DATE)); + } else if (origin_value->type() == TimeType::TIME_DATETIME) { + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DATETIME)); + } else if (origin_value->type() == TimeType::TIME_TIME) { + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_TIME)); + } + } else if constexpr (T == TYPE_DATEV2) { + auto origin_value = + reinterpret_cast*>(data); + TDateLiteral date_literal; + char convert_buffer[30]; + origin_value->to_string(convert_buffer); + date_literal.__set_value(convert_buffer); + (*node).__set_date_literal(date_literal); + (*node).__set_node_type(TExprNodeType::DATE_LITERAL); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DATEV2)); + } else if constexpr (T == TYPE_DATETIMEV2) { + auto origin_value = + reinterpret_cast*>( + data); + TDateLiteral date_literal; + char convert_buffer[30]; + origin_value->to_string(convert_buffer); + date_literal.__set_value(convert_buffer); + (*node).__set_date_literal(date_literal); + (*node).__set_node_type(TExprNodeType::DATE_LITERAL); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DATETIMEV2)); + } else if constexpr (T == TYPE_DECIMALV2) { + auto origin_value = reinterpret_cast(data); + (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL); + TDecimalLiteral decimal_literal; + decimal_literal.__set_value(origin_value->to_string()); + (*node).__set_decimal_literal(decimal_literal); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DECIMALV2, precision, scale)); + } else if constexpr (T == TYPE_DECIMAL32) { + auto origin_value = reinterpret_cast*>(data); + (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL); + TDecimalLiteral decimal_literal; + decimal_literal.__set_value(origin_value->to_string(scale)); + (*node).__set_decimal_literal(decimal_literal); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DECIMAL32, precision, scale)); + } else if constexpr (T == TYPE_DECIMAL64) { + auto origin_value = reinterpret_cast*>(data); + (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL); + TDecimalLiteral decimal_literal; + decimal_literal.__set_value(origin_value->to_string(scale)); + (*node).__set_decimal_literal(decimal_literal); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DECIMAL64, precision, scale)); + } else if constexpr (T == TYPE_DECIMAL128I) { + auto origin_value = reinterpret_cast*>(data); + (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL); + TDecimalLiteral decimal_literal; + decimal_literal.__set_value(origin_value->to_string(scale)); + (*node).__set_decimal_literal(decimal_literal); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DECIMAL128I, precision, scale)); + } else if constexpr (T == TYPE_FLOAT) { + auto origin_value = reinterpret_cast(data); + (*node).__set_node_type(TExprNodeType::FLOAT_LITERAL); + TFloatLiteral float_literal; + float_literal.__set_value(*origin_value); + (*node).__set_float_literal(float_literal); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_FLOAT)); + } else if constexpr (T == TYPE_DOUBLE) { + auto origin_value = reinterpret_cast(data); + (*node).__set_node_type(TExprNodeType::FLOAT_LITERAL); + TFloatLiteral float_literal; + float_literal.__set_value(*origin_value); + (*node).__set_float_literal(float_literal); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DOUBLE)); + } else if constexpr ((T == TYPE_STRING) || (T == TYPE_CHAR) || (T == TYPE_VARCHAR)) { + auto origin_value = reinterpret_cast(data); + (*node).__set_node_type(TExprNodeType::STRING_LITERAL); + TStringLiteral string_literal; + string_literal.__set_value(origin_value->to_string()); + (*node).__set_string_literal(string_literal); + (*node).__set_type(create_type_desc(PrimitiveType::TYPE_STRING)); + } else { + return Status::InvalidArgument("Invalid argument type!"); + } + return Status::OK(); +} + } // namespace doris diff --git a/be/src/pipeline/pipeline_task.cpp b/be/src/pipeline/pipeline_task.cpp index 87df03c83b96a5d..035db53a274e61e 100644 --- a/be/src/pipeline/pipeline_task.cpp +++ b/be/src/pipeline/pipeline_task.cpp @@ -24,7 +24,6 @@ #include -#include "common/status.h" #include "pipeline/exec/operator.h" #include "pipeline/pipeline.h" #include "pipeline_fragment_context.h" @@ -278,9 +277,6 @@ Status PipelineTask::execute(bool* eos) { if (_block->rows() != 0 || *eos) { SCOPED_TIMER(_sink_timer); auto status = _sink->sink(_state, block, _data_state); - if (status.is()) { - status = _sink->sink(_state, block, _data_state); - } if (!status.is()) { RETURN_IF_ERROR(status); } diff --git a/be/src/runtime/load_channel.cpp b/be/src/runtime/load_channel.cpp index bee2f906bc9e779..d39c72352b7bda7 100644 --- a/be/src/runtime/load_channel.cpp +++ b/be/src/runtime/load_channel.cpp @@ -86,12 +86,7 @@ Status LoadChannel::open(const PTabletWriterOpenRequest& params) { } } - if (params.is_incremental()) { - // incremental open would ensure not to open tablet repeatedly - RETURN_IF_ERROR(channel->incremental_open(params)); - } else { - RETURN_IF_ERROR(channel->open(params)); - } + RETURN_IF_ERROR(channel->open(params)); _opened = true; _last_updated_time.store(time(nullptr)); diff --git a/be/src/runtime/plan_fragment_executor.cpp b/be/src/runtime/plan_fragment_executor.cpp index 1d8db7ce5a97a21..750ff7083391f8b 100644 --- a/be/src/runtime/plan_fragment_executor.cpp +++ b/be/src/runtime/plan_fragment_executor.cpp @@ -39,7 +39,6 @@ #include "common/config.h" #include "common/logging.h" -#include "common/status.h" #include "common/version_internal.h" #include "exec/data_sink.h" #include "exec/exec_node.h" @@ -323,6 +322,7 @@ Status PlanFragmentExecutor::open_vectorized_internal() { while (!eos) { RETURN_IF_CANCELLED(_runtime_state); RETURN_IF_ERROR(get_vectorized_internal(&block, &eos)); + // Collect this plan and sub plan statistics, and send to parent plan. if (_collect_query_statistics_with_every_batch) { _collect_query_statistics(); @@ -330,13 +330,6 @@ Status PlanFragmentExecutor::open_vectorized_internal() { if (!eos || block.rows() > 0) { auto st = _sink->send(runtime_state(), &block); - //TODO: Asynchronisation need refactor this - if (st.is()) { // created partition, do it again. - st = _sink->send(runtime_state(), &block); - if (st.is()) { - LOG(WARNING) << "have to create partition again..."; - } - } if (st.is()) { break; } diff --git a/be/src/runtime/tablets_channel.cpp b/be/src/runtime/tablets_channel.cpp index 098f1eb25255e65..2e21ec92c349c3b 100644 --- a/be/src/runtime/tablets_channel.cpp +++ b/be/src/runtime/tablets_channel.cpp @@ -24,7 +24,6 @@ // IWYU pragma: no_include #include "common/compiler_util.h" // IWYU pragma: keep -#include "common/status.h" // IWYU pragma: no_include #include // IWYU pragma: keep #include @@ -119,73 +118,6 @@ Status TabletsChannel::open(const PTabletWriterOpenRequest& request) { return Status::OK(); } -Status TabletsChannel::incremental_open(const PTabletWriterOpenRequest& params) { - if (_state == kInitialized) { // haven't opened - return open(params); - } - std::lock_guard l(_lock); - std::vector* index_slots = nullptr; - int32_t schema_hash = 0; - for (auto& index : _schema->indexes()) { - if (index->index_id == _index_id) { - index_slots = &index->slots; - schema_hash = index->schema_hash; - break; - } - } - if (index_slots == nullptr) { - Status::InternalError("unknown index id, key={}", _key.to_string()); - } - // update tablets - std::vector tablet_ids; - tablet_ids.reserve(params.tablets_size()); - size_t incremental_tablet_num = 0; - std::stringstream ss; - ss << "LocalTabletsChannel txn_id: " << _txn_id << " load_id: " << print_id(params.id()) - << " incremental open delta writer: "; - - for (auto& tablet : params.tablets()) { - if (_tablet_writers.find(tablet.tablet_id()) != _tablet_writers.end()) { - continue; - } - incremental_tablet_num++; - - WriteRequest wrequest; - wrequest.index_id = params.index_id(); - wrequest.tablet_id = tablet.tablet_id(); - wrequest.schema_hash = schema_hash; - wrequest.txn_id = _txn_id; - wrequest.partition_id = tablet.partition_id(); - wrequest.load_id = params.id(); - wrequest.tuple_desc = _tuple_desc; - wrequest.slots = index_slots; - wrequest.is_high_priority = _is_high_priority; - wrequest.table_schema_param = _schema; - - DeltaWriter* writer = nullptr; - auto st = DeltaWriter::open(&wrequest, &writer, _profile, _load_id); - if (!st.ok()) { - auto err_msg = fmt::format( - "open delta writer failed, tablet_id={}" - ", txn_id={}, partition_id={}, err={}", - tablet.tablet_id(), _txn_id, tablet.partition_id(), st.to_string()); - LOG(WARNING) << err_msg; - return Status::InternalError(err_msg); - } - ss << "[" << tablet.tablet_id() << "]"; - { - std::lock_guard l(_tablet_writers_lock); - _tablet_writers.emplace(tablet.tablet_id(), writer); - } - } - - _s_tablet_writer_count += incremental_tablet_num; - LOG(INFO) << ss.str(); - - _state = kOpened; - return Status::OK(); -} - Status TabletsChannel::close( LoadChannel* parent, int sender_id, int64_t backend_id, bool* finished, const google::protobuf::RepeatedField& partition_ids, @@ -348,7 +280,7 @@ void TabletsChannel::_commit_txn(DeltaWriter* writer, void TabletsChannel::_add_error_tablet( google::protobuf::RepeatedPtrField* tablet_errors, int64_t tablet_id, - Status error) const { + Status error) { PTabletError* tablet_error = tablet_errors->Add(); tablet_error->set_tablet_id(tablet_id); tablet_error->set_msg(error.to_string()); @@ -369,15 +301,10 @@ void TabletsChannel::refresh_profile() { write_mem_usage += write_mem; int64_t flush_mem = it.second->mem_consumption(MemType::FLUSH); flush_mem_usage += flush_mem; - if (write_mem > max_tablet_write_mem_usage) { - max_tablet_write_mem_usage = write_mem; - } - if (flush_mem > max_tablet_flush_mem_usage) { - max_tablet_flush_mem_usage = flush_mem; - } - if (write_mem + flush_mem > max_tablet_mem_usage) { + if (write_mem > max_tablet_write_mem_usage) max_tablet_write_mem_usage = write_mem; + if (flush_mem > max_tablet_flush_mem_usage) max_tablet_flush_mem_usage = flush_mem; + if (write_mem + flush_mem > max_tablet_mem_usage) max_tablet_mem_usage = write_mem + flush_mem; - } } } COUNTER_SET(_memory_usage_counter, write_mem_usage + flush_mem_usage); @@ -414,12 +341,7 @@ Status TabletsChannel::_open_all_writers(const PTabletWriterOpenRequest& request } #endif - int tablet_cnt = 0; for (auto& tablet : request.tablets()) { - if (_tablet_writers.find(tablet.tablet_id()) != _tablet_writers.end()) { - continue; - } - tablet_cnt++; WriteRequest wrequest; wrequest.index_id = request.index_id(); wrequest.tablet_id = tablet.tablet_id(); @@ -448,7 +370,7 @@ Status TabletsChannel::_open_all_writers(const PTabletWriterOpenRequest& request } } _s_tablet_writer_count += _tablet_writers.size(); - DCHECK_EQ(_tablet_writers.size(), tablet_cnt); + DCHECK_EQ(_tablet_writers.size(), request.tablets_size()); return Status::OK(); } @@ -526,7 +448,7 @@ Status TabletsChannel::add_batch(const PTabletWriterAddBlockRequest& request, response->mutable_tablet_errors(); auto tablet_writer_it = _tablet_writers.find(tablet_id); if (tablet_writer_it == _tablet_writers.end()) { - return Status::InternalError("unknown tablet to append data, tablet={}", tablet_id); + return Status::InternalError("unknown tablet to append data, tablet={}"); } Status st = write_func(tablet_writer_it->second); if (!st.ok()) { diff --git a/be/src/runtime/tablets_channel.h b/be/src/runtime/tablets_channel.h index fe9c226829d794b..e3d8d87ec38668a 100644 --- a/be/src/runtime/tablets_channel.h +++ b/be/src/runtime/tablets_channel.h @@ -90,8 +90,6 @@ class TabletsChannel { ~TabletsChannel(); Status open(const PTabletWriterOpenRequest& request); - // open + open writers - Status incremental_open(const PTabletWriterOpenRequest& params); // no-op when this channel has been closed or cancelled Status add_batch(const PTabletWriterAddBlockRequest& request, @@ -130,7 +128,7 @@ class TabletsChannel { void _add_broken_tablet(int64_t tablet_id); void _add_error_tablet(google::protobuf::RepeatedPtrField* tablet_errors, - int64_t tablet_id, Status error) const; + int64_t tablet_id, Status error); bool _is_broken_tablet(int64_t tablet_id); void _init_profile(RuntimeProfile* profile); diff --git a/be/src/vec/core/block.cpp b/be/src/vec/core/block.cpp index 18d73039e502c1d..0fea95a90ec28df 100644 --- a/be/src/vec/core/block.cpp +++ b/be/src/vec/core/block.cpp @@ -757,14 +757,10 @@ Block Block::copy_block(const std::vector& column_offset) const { return columns_with_type_and_name; } -void Block::append_to_block_by_selector(MutableBlock* dst, - const IColumn::Selector& selector) const { +void Block::append_block_by_selector(MutableBlock* dst, const IColumn::Selector& selector) const { DCHECK_EQ(data.size(), dst->mutable_columns().size()); for (size_t i = 0; i < data.size(); i++) { - // FIXME: this is a quickfix. we assume that only partition functions make there some - if (!is_column_const(*data[i].column)) { - data[i].column->append_data_by_selector(dst->mutable_columns()[i], selector); - } + data[i].column->append_data_by_selector(dst->mutable_columns()[i], selector); } } diff --git a/be/src/vec/core/block.h b/be/src/vec/core/block.h index fbf15a443da2ec7..cad45ac23791925 100644 --- a/be/src/vec/core/block.h +++ b/be/src/vec/core/block.h @@ -283,7 +283,7 @@ class Block { // copy a new block by the offset column Block copy_block(const std::vector& column_offset) const; - void append_to_block_by_selector(MutableBlock* dst, const IColumn::Selector& selector) const; + void append_block_by_selector(MutableBlock* dst, const IColumn::Selector& selector) const; // need exception safety static void filter_block_internal(Block* block, const std::vector& columns_to_filter, diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp index a19dafe4393a70f..341125c89e73fbf 100644 --- a/be/src/vec/exprs/vexpr.cpp +++ b/be/src/vec/exprs/vexpr.cpp @@ -55,93 +55,6 @@ namespace doris { class RowDescriptor; class RuntimeState; -TExprNode create_texpr_node_from(const void* data, const PrimitiveType& type, int precision, - int scale) { - TExprNode node; - - switch (type) { - case TYPE_BOOLEAN: { - create_texpr_literal_node(data, &node); - break; - } - case TYPE_TINYINT: { - create_texpr_literal_node(data, &node); - break; - } - case TYPE_SMALLINT: { - create_texpr_literal_node(data, &node); - break; - } - case TYPE_INT: { - create_texpr_literal_node(data, &node); - break; - } - case TYPE_BIGINT: { - create_texpr_literal_node(data, &node); - break; - } - case TYPE_LARGEINT: { - create_texpr_literal_node(data, &node); - break; - } - case TYPE_FLOAT: { - create_texpr_literal_node(data, &node); - break; - } - case TYPE_DOUBLE: { - create_texpr_literal_node(data, &node); - break; - } - case TYPE_DATEV2: { - create_texpr_literal_node(data, &node); - break; - } - case TYPE_DATETIMEV2: { - create_texpr_literal_node(data, &node); - break; - } - case TYPE_DATE: { - create_texpr_literal_node(data, &node); - break; - } - case TYPE_DATETIME: { - create_texpr_literal_node(data, &node); - break; - } - case TYPE_DECIMALV2: { - create_texpr_literal_node(data, &node, precision, scale); - break; - } - case TYPE_DECIMAL32: { - create_texpr_literal_node(data, &node, precision, scale); - break; - } - case TYPE_DECIMAL64: { - create_texpr_literal_node(data, &node, precision, scale); - break; - } - case TYPE_DECIMAL128I: { - create_texpr_literal_node(data, &node, precision, scale); - break; - } - case TYPE_CHAR: { - create_texpr_literal_node(data, &node); - break; - } - case TYPE_VARCHAR: { - create_texpr_literal_node(data, &node); - break; - } - case TYPE_STRING: { - create_texpr_literal_node(data, &node); - break; - } - default: - DCHECK(false); - throw std::invalid_argument("Invalid type!"); - } - return node; -} } // namespace doris namespace doris::vectorized { diff --git a/be/src/vec/exprs/vexpr.h b/be/src/vec/exprs/vexpr.h index 4bd891790687f1c..b2f0fb90593c4ec 100644 --- a/be/src/vec/exprs/vexpr.h +++ b/be/src/vec/exprs/vexpr.h @@ -32,7 +32,6 @@ #include "common/factory_creator.h" #include "common/status.h" #include "runtime/define_primitive_type.h" -#include "runtime/large_int_value.h" #include "runtime/types.h" #include "udf/udf.h" #include "vec/aggregate_functions/aggregate_function.h" @@ -90,7 +89,6 @@ class VExpr { /// /// Subclasses overriding this function should call VExpr::Prepare() to recursively call /// Prepare() on the expr tree - /// row_desc used in vslot_ref and some subclass to specify column virtual Status prepare(RuntimeState* state, const RowDescriptor& row_desc, VExprContext* context); @@ -255,144 +253,4 @@ class VExpr { }; } // namespace vectorized - -template -Status create_texpr_literal_node(const void* data, TExprNode* node, int precision = 0, - int scale = 0) { - if constexpr (T == TYPE_BOOLEAN) { - auto origin_value = reinterpret_cast(data); - TBoolLiteral boolLiteral; - (*node).__set_node_type(TExprNodeType::BOOL_LITERAL); - boolLiteral.__set_value(*origin_value); - (*node).__set_bool_literal(boolLiteral); - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_BOOLEAN)); - } else if constexpr (T == TYPE_TINYINT) { - auto origin_value = reinterpret_cast(data); - (*node).__set_node_type(TExprNodeType::INT_LITERAL); - TIntLiteral intLiteral; - intLiteral.__set_value(*origin_value); - (*node).__set_int_literal(intLiteral); - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_TINYINT)); - } else if constexpr (T == TYPE_SMALLINT) { - auto origin_value = reinterpret_cast(data); - (*node).__set_node_type(TExprNodeType::INT_LITERAL); - TIntLiteral intLiteral; - intLiteral.__set_value(*origin_value); - (*node).__set_int_literal(intLiteral); - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_SMALLINT)); - } else if constexpr (T == TYPE_INT) { - auto origin_value = reinterpret_cast(data); - (*node).__set_node_type(TExprNodeType::INT_LITERAL); - TIntLiteral intLiteral; - intLiteral.__set_value(*origin_value); - (*node).__set_int_literal(intLiteral); - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_INT)); - } else if constexpr (T == TYPE_BIGINT) { - auto origin_value = reinterpret_cast(data); - (*node).__set_node_type(TExprNodeType::INT_LITERAL); - TIntLiteral intLiteral; - intLiteral.__set_value(*origin_value); - (*node).__set_int_literal(intLiteral); - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_BIGINT)); - } else if constexpr (T == TYPE_LARGEINT) { - auto origin_value = reinterpret_cast(data); - (*node).__set_node_type(TExprNodeType::LARGE_INT_LITERAL); - TLargeIntLiteral large_int_literal; - large_int_literal.__set_value(LargeIntValue::to_string(*origin_value)); - (*node).__set_large_int_literal(large_int_literal); - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_LARGEINT)); - } else if constexpr ((T == TYPE_DATE) || (T == TYPE_DATETIME) || (T == TYPE_TIME)) { - auto origin_value = reinterpret_cast(data); - TDateLiteral date_literal; - char convert_buffer[30]; - origin_value->to_string(convert_buffer); - date_literal.__set_value(convert_buffer); - (*node).__set_date_literal(date_literal); - (*node).__set_node_type(TExprNodeType::DATE_LITERAL); - if (origin_value->type() == TimeType::TIME_DATE) { - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DATE)); - } else if (origin_value->type() == TimeType::TIME_DATETIME) { - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DATETIME)); - } else if (origin_value->type() == TimeType::TIME_TIME) { - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_TIME)); - } - } else if constexpr (T == TYPE_DATEV2) { - auto origin_value = - reinterpret_cast*>(data); - TDateLiteral date_literal; - char convert_buffer[30]; - origin_value->to_string(convert_buffer); - date_literal.__set_value(convert_buffer); - (*node).__set_date_literal(date_literal); - (*node).__set_node_type(TExprNodeType::DATE_LITERAL); - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DATEV2)); - } else if constexpr (T == TYPE_DATETIMEV2) { - auto origin_value = - reinterpret_cast*>( - data); - TDateLiteral date_literal; - char convert_buffer[30]; - origin_value->to_string(convert_buffer); - date_literal.__set_value(convert_buffer); - (*node).__set_date_literal(date_literal); - (*node).__set_node_type(TExprNodeType::DATE_LITERAL); - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DATETIMEV2)); - } else if constexpr (T == TYPE_DECIMALV2) { - auto origin_value = reinterpret_cast(data); - (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL); - TDecimalLiteral decimal_literal; - decimal_literal.__set_value(origin_value->to_string()); - (*node).__set_decimal_literal(decimal_literal); - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DECIMALV2, precision, scale)); - } else if constexpr (T == TYPE_DECIMAL32) { - auto origin_value = reinterpret_cast*>(data); - (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL); - TDecimalLiteral decimal_literal; - decimal_literal.__set_value(origin_value->to_string(scale)); - (*node).__set_decimal_literal(decimal_literal); - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DECIMAL32, precision, scale)); - } else if constexpr (T == TYPE_DECIMAL64) { - auto origin_value = reinterpret_cast*>(data); - (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL); - TDecimalLiteral decimal_literal; - decimal_literal.__set_value(origin_value->to_string(scale)); - (*node).__set_decimal_literal(decimal_literal); - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DECIMAL64, precision, scale)); - } else if constexpr (T == TYPE_DECIMAL128I) { - auto origin_value = reinterpret_cast*>(data); - (*node).__set_node_type(TExprNodeType::DECIMAL_LITERAL); - TDecimalLiteral decimal_literal; - decimal_literal.__set_value(origin_value->to_string(scale)); - (*node).__set_decimal_literal(decimal_literal); - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DECIMAL128I, precision, scale)); - } else if constexpr (T == TYPE_FLOAT) { - auto origin_value = reinterpret_cast(data); - (*node).__set_node_type(TExprNodeType::FLOAT_LITERAL); - TFloatLiteral float_literal; - float_literal.__set_value(*origin_value); - (*node).__set_float_literal(float_literal); - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_FLOAT)); - } else if constexpr (T == TYPE_DOUBLE) { - auto origin_value = reinterpret_cast(data); - (*node).__set_node_type(TExprNodeType::FLOAT_LITERAL); - TFloatLiteral float_literal; - float_literal.__set_value(*origin_value); - (*node).__set_float_literal(float_literal); - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_DOUBLE)); - } else if constexpr ((T == TYPE_STRING) || (T == TYPE_CHAR) || (T == TYPE_VARCHAR)) { - auto origin_value = reinterpret_cast(data); - (*node).__set_node_type(TExprNodeType::STRING_LITERAL); - TStringLiteral string_literal; - string_literal.__set_value(origin_value->to_string()); - (*node).__set_string_literal(string_literal); - (*node).__set_type(create_type_desc(PrimitiveType::TYPE_STRING)); - } else { - return Status::InvalidArgument("Invalid argument type!"); - } - return Status::OK(); -} - -TExprNode create_texpr_node_from(const void* data, const PrimitiveType& type, int precision = 0, - int scale = 0); - } // namespace doris diff --git a/be/src/vec/exprs/vliteral.cpp b/be/src/vec/exprs/vliteral.cpp index 03d1659eee63a6f..3d39a844dc7c9e4 100644 --- a/be/src/vec/exprs/vliteral.cpp +++ b/be/src/vec/exprs/vliteral.cpp @@ -66,7 +66,6 @@ Status VLiteral::execute(VExprContext* context, vectorized::Block* block, int* r } std::string VLiteral::value() const { - //TODO: dcheck the equality of size with 1. then use string with size to replace the ss. std::stringstream out; for (size_t i = 0; i < _column_ptr->size(); i++) { if (i != 0) { diff --git a/be/src/vec/sink/vtablet_finder.cpp b/be/src/vec/sink/vtablet_finder.cpp index f1a99e260553ac0..2ee9f598b57122e 100644 --- a/be/src/vec/sink/vtablet_finder.cpp +++ b/be/src/vec/sink/vtablet_finder.cpp @@ -18,9 +18,6 @@ #include "vec/sink/vtablet_finder.h" #include -#include -#include -#include #include #include @@ -31,48 +28,37 @@ #include "common/compiler_util.h" // IWYU pragma: keep #include "common/status.h" #include "exec/tablet_info.h" -#include "exprs/runtime_filter.h" -#include "gutil/integral_types.h" #include "runtime/descriptors.h" #include "runtime/runtime_state.h" #include "vec/core/block.h" -#include "vec/core/columns_with_type_and_name.h" -#include "vec/data_types/data_type.h" -#include "vec/functions/simple_function_factory.h" namespace doris { namespace stream_load { Status OlapTabletFinder::find_tablet(RuntimeState* state, vectorized::Block* block, int row_index, const VOlapTablePartition** partition, uint32_t& tablet_index, - bool& stop_processing, bool& is_continue, - bool* missing_partition) { + bool& stop_processing, bool& is_continue) { Status status = Status::OK(); *partition = nullptr; tablet_index = 0; BlockRow block_row; block_row = {block, row_index}; if (!_vpartition->find_partition(&block_row, partition)) { - if (missing_partition != nullptr) { // auto partition table - *missing_partition = true; - return status; - } else { - RETURN_IF_ERROR(state->append_error_msg_to_file( - []() -> std::string { return ""; }, - [&]() -> std::string { - fmt::memory_buffer buf; - fmt::format_to(buf, "no partition for this tuple. tuple={}", - block->dump_data(row_index, 1)); - return fmt::to_string(buf); - }, - &stop_processing)); - _num_filtered_rows++; - if (stop_processing) { - return Status::EndOfFile("Encountered unqualified data, stop processing"); - } - is_continue = true; - return status; + RETURN_IF_ERROR(state->append_error_msg_to_file( + []() -> std::string { return ""; }, + [&]() -> std::string { + fmt::memory_buffer buf; + fmt::format_to(buf, "no partition for this tuple. tuple={}", + block->dump_data(row_index, 1)); + return fmt::to_string(buf); + }, + &stop_processing)); + _num_filtered_rows++; + if (stop_processing) { + return Status::EndOfFile("Encountered unqualified data, stop processing"); } + is_continue = true; + return status; } if (!(*partition)->is_mutable) { _num_immutable_partition_filtered_rows++; diff --git a/be/src/vec/sink/vtablet_finder.h b/be/src/vec/sink/vtablet_finder.h index 9fe64944300f35b..97282e403aa32f9 100644 --- a/be/src/vec/sink/vtablet_finder.h +++ b/be/src/vec/sink/vtablet_finder.h @@ -41,7 +41,7 @@ class OlapTabletFinder { Status find_tablet(RuntimeState* state, vectorized::Block* block, int row_index, const VOlapTablePartition** partition, uint32_t& tablet_index, - bool& filtered, bool& is_continue, bool* missing_partition = nullptr); + bool& filtered, bool& is_continue); bool is_find_tablet_every_sink() { return _find_tablet_mode == FindTabletMode::FIND_TABLET_EVERY_SINK; diff --git a/be/src/vec/sink/vtablet_sink.cpp b/be/src/vec/sink/vtablet_sink.cpp index 824ed84e0331e04..09bcdc6e25bf36e 100644 --- a/be/src/vec/sink/vtablet_sink.cpp +++ b/be/src/vec/sink/vtablet_sink.cpp @@ -25,44 +25,23 @@ #include #include #include -#include -#include -#include -#include #include #include #include #include -#include #include #include #include #include #include -#include -#include -#include -#include #include #include #include -#include #include -#include #include #include -#include "runtime/datetime_value.h" -#include "util/runtime_profile.h" -#include "vec/core/columns_with_type_and_name.h" -#include "vec/data_types/data_type.h" -#include "vec/data_types/data_type_factory.hpp" -#include "vec/data_types/data_type_string.h" -#include "vec/exprs/vexpr_fwd.h" -#include "vec/functions/simple_function_factory.h" -#include "vec/runtime/vdatetime_value.h" - #ifdef DEBUG #include #endif @@ -73,7 +52,6 @@ #include "common/object_pool.h" #include "common/status.h" #include "exec/tablet_info.h" -#include "runtime/client_cache.h" #include "runtime/define_primitive_type.h" #include "runtime/descriptors.h" #include "runtime/exec_env.h" @@ -92,7 +70,6 @@ #include "util/telemetry/telemetry.h" #include "util/thread.h" #include "util/threadpool.h" -#include "util/thrift_rpc_helper.h" #include "util/thrift_util.h" #include "util/time.h" #include "util/uid_util.h" @@ -124,7 +101,6 @@ class TExpr; namespace stream_load { -// an IndexChannel is related to specific table and its rollup and mv class IndexChannel { public: IndexChannel(VOlapTableSink* parent, int64_t index_id, @@ -135,7 +111,6 @@ class IndexChannel { } ~IndexChannel() = default; - // allow to init multi times, for incremental open more tablets for one index(table) Status init(RuntimeState* state, const std::vector& tablets); void for_each_node_channel( @@ -207,23 +182,20 @@ class IndexChannel { Status IndexChannel::init(RuntimeState* state, const std::vector& tablets) { SCOPED_CONSUME_MEM_TRACKER(_index_channel_tracker.get()); for (auto& tablet : tablets) { - // First find the location BEs of this tablet - auto tablet_locations = _parent->_location->find_tablet(tablet.tablet_id); - if (tablet_locations == nullptr) { + auto location = _parent->_location->find_tablet(tablet.tablet_id); + if (location == nullptr) { return Status::InternalError("unknown tablet, tablet_id={}", tablet.tablet_id); } std::vector> channels; - // For tablet, deal with its' all replica (in some node). - for (auto& replica_node_id : tablet_locations->node_ids) { + for (auto& node_id : location->node_ids) { std::shared_ptr channel; - auto it = _node_channels.find(replica_node_id); - // when we prepare for TableSink or incremental open tablet, we need init + auto it = _node_channels.find(node_id); if (it == _node_channels.end()) { // NodeChannel is not added to the _parent->_pool. // Because the deconstruction of NodeChannel may take a long time to wait rpc finish. // but the ObjectPool will hold a spin lock to delete objects. - channel = std::make_shared(_parent, this, replica_node_id); - _node_channels.emplace(replica_node_id, channel); + channel = std::make_shared(_parent, this, node_id); + _node_channels.emplace(node_id, channel); } else { channel = it->second; } @@ -235,7 +207,7 @@ Status IndexChannel::init(RuntimeState* state, const std::vector(fmt::format( "NodeChannel:indexID={}:threadId={}", std::to_string(_index_channel->_index_id), thread_context()->get_thread_id())); } VNodeChannel::~VNodeChannel() { - for (auto& closure : _open_closures) { - if (closure != nullptr) { - if (closure->unref()) { - delete closure; - } - closure = nullptr; + if (_open_closure != nullptr) { + if (_open_closure->unref()) { + delete _open_closure; } + _open_closure = nullptr; } if (_add_block_closure != nullptr) { delete _add_block_closure; _add_block_closure = nullptr; } + if (_open_closure != nullptr) { + delete _open_closure; + } static_cast(_cur_add_block_request.release_id()); } @@ -369,12 +338,12 @@ Status VNodeChannel::init(RuntimeState* state) { SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); _tuple_desc = _parent->_output_tuple_desc; _state = state; - // get corresponding BE node. auto node = _parent->_nodes_info->find_node(_node_id); if (node == nullptr) { _cancelled = true; return Status::InternalError("unknown node id, id={}", _node_id); } + _node_info = *node; _load_info = "load_id=" + print_id(_parent->_load_id) + @@ -395,9 +364,7 @@ Status VNodeChannel::init(RuntimeState* state) { _timeout_watch.start(); // Initialize _cur_add_block_request - if (!_cur_add_block_request.has_id()) { - _cur_add_block_request.set_allocated_id(&_parent->_load_id); - } + _cur_add_block_request.set_allocated_id(&_parent->_load_id); _cur_add_block_request.set_index_id(_index_channel->_index_id); _cur_add_block_request.set_sender_id(_parent->_sender_id); _cur_add_block_request.set_backend_id(_node_id); @@ -414,22 +381,17 @@ Status VNodeChannel::init(RuntimeState* state) { return Status::OK(); } -void VNodeChannel::_open_internal(bool is_incremental) { +void VNodeChannel::open() { SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); PTabletWriterOpenRequest request; request.set_allocated_id(&_parent->_load_id); request.set_index_id(_index_channel->_index_id); request.set_txn_id(_parent->_txn_id); request.set_allocated_schema(_parent->_schema->to_protobuf()); - std::set deduper; for (auto& tablet : _all_tablets) { - if (deduper.contains(tablet.tablet_id)) { - continue; - } auto ptablet = request.add_tablets(); ptablet->set_partition_id(tablet.partition_id); ptablet->set_tablet_id(tablet.tablet_id); - deduper.insert(tablet.tablet_id); } request.set_num_senders(_parent->_num_senders); request.set_need_gen_rollup(false); // Useless but it is a required field in pb @@ -440,75 +402,160 @@ void VNodeChannel::_open_internal(bool is_incremental) { request.set_is_vectorized(true); request.set_backend_id(_node_id); request.set_enable_profile(_state->enable_profile()); - request.set_is_incremental(is_incremental); - auto* open_closure = new RefCountClosure {}; - open_closure->ref(); + _open_closure = new RefCountClosure(); + _open_closure->ref(); - open_closure->ref(); // This ref is for RPC's reference - open_closure->cntl.set_timeout_ms(config::tablet_writer_open_rpc_timeout_sec * 1000); + // This ref is for RPC's reference + _open_closure->ref(); + _open_closure->cntl.set_timeout_ms(config::tablet_writer_open_rpc_timeout_sec * 1000); if (config::tablet_writer_ignore_eovercrowded) { - open_closure->cntl.ignore_eovercrowded(); + _open_closure->cntl.ignore_eovercrowded(); } - // the real transmission here. the corresponding BE's load mgr will open load channel for it. - _stub->tablet_writer_open(&open_closure->cntl, &request, &open_closure->result, open_closure); - + _stub->tablet_writer_open(&_open_closure->cntl, &request, &_open_closure->result, + _open_closure); static_cast(request.release_id()); static_cast(request.release_schema()); } -void VNodeChannel::open() { - _open_internal(false); -} - -void VNodeChannel::incremental_open() { - _open_internal(true); -} - Status VNodeChannel::open_wait() { - Status status; - for (auto& open_closure : _open_closures) { - open_closure->join(); - SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); - if (open_closure->cntl.Failed()) { - if (!ExecEnv::GetInstance()->brpc_internal_client_cache()->available( - _stub, _node_info.host, _node_info.brpc_port)) { - ExecEnv::GetInstance()->brpc_internal_client_cache()->erase( - open_closure->cntl.remote_side()); - } - - _cancelled = true; - auto error_code = open_closure->cntl.ErrorCode(); - auto error_text = open_closure->cntl.ErrorText(); - if (open_closure->unref()) { - delete open_closure; - } - open_closure = nullptr; - return Status::InternalError( - "failed to open tablet writer, error={}, error_text={}, info={}", - berror(error_code), error_text, channel_info()); - } - status = Status::create(open_closure->result.status()); - if (open_closure->unref()) { - delete open_closure; + _open_closure->join(); + SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); + if (_open_closure->cntl.Failed()) { + if (!ExecEnv::GetInstance()->brpc_internal_client_cache()->available( + _stub, _node_info.host, _node_info.brpc_port)) { + ExecEnv::GetInstance()->brpc_internal_client_cache()->erase( + _open_closure->cntl.remote_side()); } - open_closure = nullptr; - if (!status.ok()) { - _cancelled = true; - return status; + _cancelled = true; + auto error_code = _open_closure->cntl.ErrorCode(); + auto error_text = _open_closure->cntl.ErrorText(); + if (_open_closure->unref()) { + delete _open_closure; } + _open_closure = nullptr; + return Status::InternalError( + "failed to open tablet writer, error={}, error_text={}, info={}", + berror(error_code), error_text, channel_info()); + } + Status status(Status::create(_open_closure->result.status())); + if (_open_closure->unref()) { + delete _open_closure; + } + _open_closure = nullptr; + + if (!status.ok()) { + _cancelled = true; + return status; } // add block closure _add_block_closure = ReusableClosure::create(); - _add_block_closure->addFailedHandler( - [this](bool is_last_rpc) { _add_block_failed_callback(is_last_rpc); }); + _add_block_closure->addFailedHandler([this](bool is_last_rpc) { + std::lock_guard l(this->_closed_lock); + if (this->_is_closed) { + // if the node channel is closed, no need to call `mark_as_failed`, + // and notice that _index_channel may already be destroyed. + return; + } + SCOPED_ATTACH_TASK(_state); + // If rpc failed, mark all tablets on this node channel as failed + _index_channel->mark_as_failed(this->node_id(), this->host(), + fmt::format("rpc failed, error coed:{}, error text:{}", + _add_block_closure->cntl.ErrorCode(), + _add_block_closure->cntl.ErrorText()), + -1); + Status st = _index_channel->check_intolerable_failure(); + if (!st.ok()) { + _cancel_with_msg(fmt::format("{}, err: {}", channel_info(), st.to_string())); + } else if (is_last_rpc) { + // if this is last rpc, will must set _add_batches_finished. otherwise, node channel's close_wait + // will be blocked. + _add_batches_finished = true; + } + }); + + _add_block_closure->addSuccessHandler([this](const PTabletWriterAddBlockResult& result, + bool is_last_rpc) { + std::lock_guard l(this->_closed_lock); + if (this->_is_closed) { + // if the node channel is closed, no need to call the following logic, + // and notice that _index_channel may already be destroyed. + return; + } + SCOPED_ATTACH_TASK(_state); + Status status(Status::create(result.status())); + if (status.ok()) { + // if has error tablet, handle them first + for (auto& error : result.tablet_errors()) { + _index_channel->mark_as_failed(this->node_id(), this->host(), + "tablet error: " + error.msg(), error.tablet_id()); + } - _add_block_closure->addSuccessHandler( - [this](const PTabletWriterAddBlockResult& result, bool is_last_rpc) { - _add_block_success_callback(result, is_last_rpc); - }); + Status st = _index_channel->check_intolerable_failure(); + if (!st.ok()) { + _cancel_with_msg(st.to_string()); + } else if (is_last_rpc) { + for (auto& tablet : result.tablet_vec()) { + TTabletCommitInfo commit_info; + commit_info.tabletId = tablet.tablet_id(); + commit_info.backendId = _node_id; + _tablet_commit_infos.emplace_back(std::move(commit_info)); + if (tablet.has_received_rows()) { + _tablets_received_rows.emplace_back(tablet.tablet_id(), + tablet.received_rows()); + } + if (tablet.has_num_rows_filtered()) { + _state->update_num_rows_filtered_in_strict_mode_partial_update( + tablet.num_rows_filtered()); + } + VLOG_CRITICAL << "master replica commit info: tabletId=" << tablet.tablet_id() + << ", backendId=" << _node_id + << ", master node id: " << this->node_id() + << ", host: " << this->host() << ", txn_id=" << _parent->_txn_id; + } + if (_parent->_write_single_replica) { + for (auto& tablet_slave_node_ids : result.success_slave_tablet_node_ids()) { + for (auto slave_node_id : tablet_slave_node_ids.second.slave_node_ids()) { + TTabletCommitInfo commit_info; + commit_info.tabletId = tablet_slave_node_ids.first; + commit_info.backendId = slave_node_id; + _tablet_commit_infos.emplace_back(std::move(commit_info)); + VLOG_CRITICAL << "slave replica commit info: tabletId=" + << tablet_slave_node_ids.first + << ", backendId=" << slave_node_id + << ", master node id: " << this->node_id() + << ", host: " << this->host() + << ", txn_id=" << _parent->_txn_id; + } + } + } + _add_batches_finished = true; + } + } else { + _cancel_with_msg(fmt::format("{}, add batch req success but status isn't ok, err: {}", + channel_info(), status.to_string())); + } + + if (result.has_execution_time_us()) { + _add_batch_counter.add_batch_execution_time_us += result.execution_time_us(); + _add_batch_counter.add_batch_wait_execution_time_us += result.wait_execution_time_us(); + _add_batch_counter.add_batch_num++; + } + if (result.has_load_channel_profile()) { + TRuntimeProfileTree tprofile; + const uint8_t* buf = (const uint8_t*)result.load_channel_profile().data(); + uint32_t len = result.load_channel_profile().size(); + auto st = deserialize_thrift_msg(buf, &len, false, &tprofile); + if (st.ok()) { + _state->load_channel_profile()->update(tprofile); + } else { + LOG(WARNING) << "load channel TRuntimeProfileTree deserialize failed, errmsg=" + << st; + } + } + }); return status; } @@ -592,7 +639,7 @@ Status VNodeChannel::add_block(vectorized::Block* block, const Payload* payload, SCOPED_RAW_TIMER(&_stat.append_node_channel_ns); if (is_append) { // Do not split the data of the block by tablets but append it to a single delta writer. - // This is a faster way to send block than append_to_block_by_selector + // This is a faster way to send block than append_block_by_selector // TODO: we could write to local delta writer if single_replica_load is true VLOG_DEBUG << "send whole block by append block"; std::vector tablets(block->rows(), payload->second[0]); @@ -601,12 +648,12 @@ Status VNodeChannel::add_block(vectorized::Block* block, const Payload* payload, columns.reserve(block->columns()); // Hold the reference of block columns to avoid copying for (auto column : block->get_columns()) { - columns.push_back(std::move(*column).mutate()); + columns.push_back(column->assume_mutable()); } *_cur_add_block_request.mutable_tablet_ids() = {tablets.begin(), tablets.end()}; _cur_add_block_request.set_is_single_tablet_block(true); } else { - block->append_to_block_by_selector(_cur_mutable_block.get(), *(payload->first)); + block->append_block_by_selector(_cur_mutable_block.get(), *(payload->first)); for (auto tablet_id : payload->second) { _cur_add_block_request.add_tablet_ids(tablet_id); } @@ -619,8 +666,6 @@ Status VNodeChannel::add_block(vectorized::Block* block, const Payload* payload, std::lock_guard l(_pending_batches_lock); // To simplify the add_row logic, postpone adding block into req until the time of sending req _pending_batches_bytes += _cur_mutable_block->allocated_bytes(); - _cur_add_block_request.set_eos( - false); // for multi-add, only when marking close we set it eos. _pending_blocks.emplace(std::move(_cur_mutable_block), _cur_add_block_request); _pending_batches_num++; VLOG_DEBUG << "VOlapTableSink:" << _parent << " VNodeChannel:" << this @@ -650,7 +695,7 @@ int VNodeChannel::try_send_and_fetch_status(RuntimeState* state, // We are sure that try_send_batch is not running if (_pending_batches_num > 0) { auto s = thread_pool_token->submit_func( - std::bind(&VNodeChannel::try_send_pending_block, this, state)); + std::bind(&VNodeChannel::try_send_block, this, state)); if (!s.ok()) { _cancel_with_msg("submit send_batch task to send_batch_thread_pool failed"); // clear in flight @@ -691,12 +736,13 @@ Status VNodeChannel::none_of(std::initializer_list vars) { return st; } -void VNodeChannel::try_send_pending_block(RuntimeState* state) { +void VNodeChannel::try_send_block(RuntimeState* state) { SCOPED_ATTACH_TASK(state); SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker); SCOPED_ATOMIC_TIMER(&_actual_consume_ns); AddBlockReq send_block; { + debug::ScopedTSANIgnoreReadsAndWrites ignore_tsan; std::lock_guard l(_pending_batches_lock); DCHECK(!_pending_blocks.empty()); send_block = std::move(_pending_blocks.front()); @@ -821,108 +867,6 @@ void VNodeChannel::try_send_pending_block(RuntimeState* state) { _next_packet_seq++; } -void VNodeChannel::_add_block_success_callback(const PTabletWriterAddBlockResult& result, - bool is_last_rpc) { - std::lock_guard l(this->_closed_lock); - if (this->_is_closed) { - // if the node channel is closed, no need to call the following logic, - // and notice that _index_channel may already be destroyed. - return; - } - SCOPED_ATTACH_TASK(_state); - Status status(Status::create(result.status())); - if (status.ok()) { - // if has error tablet, handle them first - for (auto& error : result.tablet_errors()) { - _index_channel->mark_as_failed(this->node_id(), this->host(), - "tablet error: " + error.msg(), error.tablet_id()); - } - - Status st = _index_channel->check_intolerable_failure(); - if (!st.ok()) { - _cancel_with_msg(st.to_string()); - } else if (is_last_rpc) { - for (auto& tablet : result.tablet_vec()) { - TTabletCommitInfo commit_info; - commit_info.tabletId = tablet.tablet_id(); - commit_info.backendId = _node_id; - _tablet_commit_infos.emplace_back(std::move(commit_info)); - if (tablet.has_received_rows()) { - _tablets_received_rows.emplace_back(tablet.tablet_id(), tablet.received_rows()); - } - if (tablet.has_num_rows_filtered()) { - _state->update_num_rows_filtered_in_strict_mode_partial_update( - tablet.num_rows_filtered()); - } - VLOG_CRITICAL << "master replica commit info: tabletId=" << tablet.tablet_id() - << ", backendId=" << _node_id - << ", master node id: " << this->node_id() - << ", host: " << this->host() << ", txn_id=" << _parent->_txn_id; - } - if (_parent->_write_single_replica) { - for (auto& tablet_slave_node_ids : result.success_slave_tablet_node_ids()) { - for (auto slave_node_id : tablet_slave_node_ids.second.slave_node_ids()) { - TTabletCommitInfo commit_info; - commit_info.tabletId = tablet_slave_node_ids.first; - commit_info.backendId = slave_node_id; - _tablet_commit_infos.emplace_back(std::move(commit_info)); - VLOG_CRITICAL - << "slave replica commit info: tabletId=" - << tablet_slave_node_ids.first << ", backendId=" << slave_node_id - << ", master node id: " << this->node_id() - << ", host: " << this->host() << ", txn_id=" << _parent->_txn_id; - } - } - } - _add_batches_finished = true; - } - } else { - _cancel_with_msg(fmt::format("{}, add batch req success but status isn't ok, err: {}", - channel_info(), status.to_string())); - } - - if (result.has_execution_time_us()) { - _add_batch_counter.add_batch_execution_time_us += result.execution_time_us(); - _add_batch_counter.add_batch_wait_execution_time_us += result.wait_execution_time_us(); - _add_batch_counter.add_batch_num++; - } - if (result.has_load_channel_profile()) { - TRuntimeProfileTree tprofile; - const uint8_t* buf = (const uint8_t*)result.load_channel_profile().data(); - uint32_t len = result.load_channel_profile().size(); - auto st = deserialize_thrift_msg(buf, &len, false, &tprofile); - if (st.ok()) { - _state->load_channel_profile()->update(tprofile); - } else { - LOG(WARNING) << "load channel TRuntimeProfileTree deserialize failed, errmsg=" << st; - } - } -} - -void VNodeChannel::_add_block_failed_callback(bool is_last_rpc) { - std::lock_guard l(this->_closed_lock); - if (this->_is_closed) { - // if the node channel is closed, no need to call `mark_as_failed`, - // and notice that _index_channel may already be destroyed. - return; - } - SCOPED_ATTACH_TASK(_state); - // If rpc failed, mark all tablets on this node channel as failed - _index_channel->mark_as_failed( - this->node_id(), this->host(), - fmt::format("rpc failed, error coed:{}, error text:{}", - _add_block_closure->cntl.ErrorCode(), _add_block_closure->cntl.ErrorText()), - -1); - Status st = _index_channel->check_intolerable_failure(); - if (!st.ok()) { - _cancel_with_msg(fmt::format("{}, err: {}", channel_info(), st.to_string())); - } else if (is_last_rpc) { - // if this is last rpc, will must set _add_batches_finished. otherwise, node channel's close_wait - // will be blocked. - _add_batches_finished = true; - } -} - void VNodeChannel::cancel(const std::string& cancel_msg) { if (_is_closed) { // skip the channels that have been canceled or close_wait. @@ -984,6 +928,7 @@ Status VNodeChannel::close_wait(RuntimeState* state) { // waiting for finished, it may take a long time, so we couldn't set a timeout // In pipeline, is_close_done() is false at this time, will not bock. while (!_add_batches_finished && !_cancelled && !state->is_cancelled()) { + // std::this_thread::sleep_for(std::chrono::milliseconds(1)); bthread_usleep(1000); } _close_time_ms = UnixMillis() - _close_time_ms; @@ -1016,12 +961,12 @@ void VNodeChannel::mark_close() { _cur_add_block_request.set_eos(true); { + debug::ScopedTSANIgnoreReadsAndWrites ignore_tsan; std::lock_guard l(_pending_batches_lock); if (!_cur_mutable_block) { // add a dummy block _cur_mutable_block = vectorized::MutableBlock::create_unique(); } - // when prepare to close, add block to queue so that try_send_pending_block thread will send it. _pending_blocks.emplace(std::move(_cur_mutable_block), _cur_add_block_request); _pending_batches_num++; DCHECK(_pending_blocks.back().second.eos()); @@ -1135,8 +1080,6 @@ Status VOlapTableSink::prepare(RuntimeState* state) { _filter_timer = ADD_CHILD_TIMER(_profile, "FilterTime", "SendDataTime"); _where_clause_timer = ADD_CHILD_TIMER(_profile, "WhereClauseTime", "SendDataTime"); _append_node_channel_timer = ADD_CHILD_TIMER(_profile, "AppendNodeChannelTime", "SendDataTime"); - _add_partition_request_timer = - ADD_CHILD_TIMER(_profile, "AddPartitionRequestTime", "SendDataTime"); _validate_data_timer = ADD_TIMER(_profile, "ValidateDataTime"); _open_timer = ADD_TIMER(_profile, "OpenTime"); _close_timer = ADD_TIMER(_profile, "CloseWaitTime"); @@ -1183,22 +1126,15 @@ Status VOlapTableSink::prepare(RuntimeState* state) { tablets.emplace_back(std::move(tablet_with_partition)); } } - if (tablets.empty() && !_vpartition->is_auto_partition()) { + if (UNLIKELY(tablets.empty())) { LOG(WARNING) << "load job:" << state->load_job_id() << " index: " << index->index_id << " would open 0 tablet"; } _channels.emplace_back(new IndexChannel(this, index->index_id, index->where_clause)); - _index_id_to_channel[index->index_id] = _channels.back(); RETURN_IF_ERROR(_channels.back()->init(state, tablets)); } // Prepare the exprs to run. RETURN_IF_ERROR(vectorized::VExpr::prepare(_output_vexpr_ctxs, state, _row_desc)); - // prepare for auto partition functions - if (_vpartition->is_auto_partition()) { - auto [part_ctx, part_func] = _get_partition_function(); - RETURN_IF_ERROR(part_func->prepare(_state, *_output_row_desc, part_ctx.get())); - } - _prepare = true; return Status::OK(); } @@ -1245,7 +1181,6 @@ Status VOlapTableSink::open(RuntimeState* state) { MIN(_send_batch_parallelism, config::max_send_batch_parallelism_per_job); _send_batch_thread_pool_token = state->exec_env()->send_batch_thread_pool()->new_token( ThreadPool::ExecutionMode::CONCURRENT, send_batch_parallelism); - // start to send batch continually if (bthread_start_background(&_sender_thread, nullptr, periodic_send_batch, (void*)this) != 0) { return Status::Error("bthread_start_backgroud failed"); } @@ -1257,12 +1192,7 @@ void VOlapTableSink::_send_batch_process() { SCOPED_TIMER(_non_blocking_send_timer); SCOPED_ATTACH_TASK(_state); SCOPED_CONSUME_MEM_TRACKER(_mem_tracker); - - bool had_effect = false; while (true) { - // incremental open will temporarily make channels into abnormal state. stop checking when this. - std::unique_lock l(_stop_check_channel); - int running_channels_num = 0; for (const auto& index_channel : _channels) { index_channel->for_each_node_channel([&running_channels_num, @@ -1272,14 +1202,11 @@ void VOlapTableSink::_send_batch_process() { }); } - // if there is no channel, maybe auto partition table. so check does there have had running channels ever. - if (running_channels_num == 0 && had_effect) { + if (running_channels_num == 0) { LOG(INFO) << "all node channels are stopped(maybe finished/offending/cancelled), " "sender thread exit. " << print_id(_load_id); return; - } else if (running_channels_num != 0) { - had_effect = true; } bthread_usleep(config::olap_table_sink_send_interval_ms * 1000); } @@ -1293,96 +1220,6 @@ size_t VOlapTableSink::get_pending_bytes() const { return mem_consumption; } -Status VOlapTableSink::_automatic_create_partition() { - SCOPED_TIMER(_add_partition_request_timer); - TCreatePartitionRequest request; - TCreatePartitionResult result; - request.__set_txn_id(_txn_id); - request.__set_db_id(_vpartition->db_id()); - request.__set_table_id(_vpartition->table_id()); - request.__set_partitionValues(_partitions_need_create); - - VLOG(1) << "automatic partition rpc begin request " << request; - TNetworkAddress master_addr = ExecEnv::GetInstance()->master_info()->network_address; - int time_out = _state->execution_timeout() * 1000; - RETURN_IF_ERROR(ThriftRpcHelper::rpc( - master_addr.hostname, master_addr.port, - [&request, &result](FrontendServiceConnection& client) { - client->createPartition(result, request); - }, - time_out)); - - Status status(Status::create(result.status)); - VLOG(1) << "automatic partition rpc end response " << result; - if (result.status.status_code == TStatusCode::OK) { - // add new created partitions - RETURN_IF_ERROR(_vpartition->add_partitions(result.partitions)); - - // add new tablet locations. it will use by address. so add to pool - auto* new_locations = _pool->add(new std::vector(result.tablets)); - _location->add_locations(*new_locations); - - // update new node info - _nodes_info->add_nodes(result.nodes); - - // incremental open node channel - RETURN_IF_ERROR(_incremental_open_node_channel(result.partitions)); - } - - return status; -} - -Status VOlapTableSink::_incremental_open_node_channel( - const std::vector& partitions) { - // do what we did in prepare() for partitions. indexes which don't change when we create new partition is orthogonal to partitions. - std::unique_lock _l(_stop_check_channel); - for (int i = 0; i < _schema->indexes().size(); ++i) { - const OlapTableIndexSchema* index = _schema->indexes()[i]; - std::vector tablets; - for (auto& t_part : partitions) { - VOlapTablePartition* part = nullptr; - RETURN_IF_ERROR(_vpartition->generate_partition_from(t_part, part)); - for (const auto& tablet : part->indexes[i].tablets) { - TTabletWithPartition tablet_with_partition; - tablet_with_partition.partition_id = part->id; - tablet_with_partition.tablet_id = tablet; - tablets.emplace_back(std::move(tablet_with_partition)); - } - DCHECK(!tablets.empty()) << "incremental open got nothing!"; - } - // update and reinit for existing channels. - std::shared_ptr channel = _index_id_to_channel[index->index_id]; - DCHECK(channel != nullptr); - RETURN_IF_ERROR(channel->init(_state, tablets)); // add tablets into it - } - - fmt::memory_buffer buf; - for (auto& channel : _channels) { - // incremental open new partition's tablet on storage side - channel->for_each_node_channel( - [](const std::shared_ptr& ch) { ch->incremental_open(); }); - fmt::format_to(buf, "index id:{}", channel->_index_id); - VLOG_DEBUG << "list of open index id = " << fmt::to_string(buf); - - channel->for_each_node_channel([&channel](const std::shared_ptr& ch) { - auto st = ch->open_wait(); - if (!st.ok()) { - // The open() phase is mainly to generate DeltaWriter instances on the nodes corresponding to each node channel. - // This phase will not fail due to a single tablet. - // Therefore, if the open() phase fails, all tablets corresponding to the node need to be marked as failed. - channel->mark_as_failed( - ch->node_id(), ch->host(), - fmt::format("{}, open failed, err: {}", ch->channel_info(), st.to_string()), - -1); - } - }); - - RETURN_IF_ERROR(channel->check_intolerable_failure()); - } - - return Status::OK(); -} - void VOlapTableSink::_generate_row_distribution_payload( ChannelDistributionPayload& channel_to_payload, const VOlapTablePartition* partition, uint32_t tablet_index, int row_idx, size_t row_cnt) { @@ -1456,27 +1293,6 @@ Status VOlapTableSink::_single_partition_generate(RuntimeState* state, vectorize return Status::OK(); } -std::pair -VOlapTableSink::_get_partition_function() { - return {_vpartition->get_part_func_ctx(), _vpartition->get_partition_function()}; -} - -void VOlapTableSink::_save_missing_values(vectorized::ColumnPtr col, - vectorized::DataTypePtr value_type, - std::vector filter) { - _partitions_need_create.clear(); - std::set deduper; - // de-duplication - for (auto row : filter) { - deduper.emplace(value_type->to_string(*col, row)); - } - for (auto& value : deduper) { - TStringLiteral node; - node.value = value; - _partitions_need_create.emplace_back(std::vector {node}); // only 1 partition column now - } -} - Status VOlapTableSink::send(RuntimeState* state, vectorized::Block* input_block, bool eos) { SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); Status status = Status::OK(); @@ -1504,6 +1320,9 @@ Status VOlapTableSink::send(RuntimeState* state, vectorized::Block* input_block, RETURN_IF_ERROR(_block_convertor->validate_and_convert_block( state, input_block, block, _output_vexpr_ctxs, rows, eos, has_filtered_rows)); + // clear and release the references of columns + input_block->clear(); + SCOPED_RAW_TIMER(&_send_data_ns); // This is just for passing compilation. bool stop_processing = false; @@ -1513,106 +1332,30 @@ Status VOlapTableSink::send(RuntimeState* state, vectorized::Block* input_block, _row_distribution_watch.start(); auto num_rows = block->rows(); size_t partition_num = _vpartition->get_partitions().size(); - if (!_vpartition->is_auto_partition() && partition_num == 1 && - _tablet_finder->is_find_tablet_every_sink()) { + if (partition_num == 1 && _tablet_finder->is_find_tablet_every_sink()) { RETURN_IF_ERROR(_single_partition_generate(state, block.get(), channel_to_payload, num_rows, has_filtered_rows)); } else { - // if there's projection of partition calc, we need to calc it first. - auto [part_ctx, part_func] = _get_partition_function(); - int result_idx; - if (_vpartition->is_projection_partition()) { - // calc the start value of missing partition ranges. - part_func->execute(part_ctx.get(), block.get(), &result_idx); - VLOG_DEBUG << "Partition-calculated block:" << block->dump_data(); - // change the column to compare to transformed. - _vpartition->set_transformed_slots({(uint16_t)result_idx}); - } - - if (_vpartition->is_auto_partition()) { - std::vector partition_keys = _vpartition->get_partition_keys(); - //TODO: use loop to create missing_vals for multi column. - CHECK(partition_keys.size() == 1) - << "now support only 1 partition column for auto partitions."; - auto partition_col = block->get_by_position(partition_keys[0]); - - std::vector missing_map; // indice of missing values in partition_col - missing_map.reserve(partition_col.column->size()); - - // try to find tablet and save missing value - for (int i = 0; i < num_rows; ++i) { - if (UNLIKELY(has_filtered_rows) && _block_convertor->filter_bitmap().Get(i)) { - continue; - } - const VOlapTablePartition* partition = nullptr; - bool is_continue = false; - uint32_t tablet_index = 0; - bool missing_this = false; - RETURN_IF_ERROR(_tablet_finder->find_tablet(state, block.get(), i, &partition, - tablet_index, stop_processing, - is_continue, &missing_this)); - if (missing_this) { - missing_map.push_back(i); - } else { - _generate_row_distribution_payload(channel_to_payload, partition, tablet_index, - i, 1); - } + for (int i = 0; i < num_rows; ++i) { + if (UNLIKELY(has_filtered_rows) && _block_convertor->filter_bitmap().Get(i)) { + continue; } - missing_map.shrink_to_fit(); - - // for missing partition keys, calc the missing partition and save in _partitions_need_create - auto type = partition_col.type; - if (missing_map.size() > 0) { - auto return_type = part_func->data_type(); - - // expose the data column - vectorized::ColumnPtr range_left_col = block->get_by_position(result_idx).column; - if (auto* nullable = - check_and_get_column(*range_left_col)) { - range_left_col = nullable->get_nested_column_ptr(); - return_type = - assert_cast(return_type.get()) - ->get_nested_type(); - } - // calc the end value and save them. - _save_missing_values(range_left_col, return_type, missing_map); - // then call FE to create it. then FragmentExecutor will redo the load. - RETURN_IF_ERROR(_automatic_create_partition()); - // now we need to rollback the metrics - _number_input_rows -= rows; - state->update_num_rows_load_total(-rows); - state->update_num_bytes_load_total(-bytes); - DorisMetrics::instance()->load_rows->increment(-rows); - DorisMetrics::instance()->load_bytes->increment(-bytes); - // In the next round, we will _generate_row_distribution_payload again to get right payload of new tablet - LOG(INFO) << "Auto created partition. Send block again."; - return Status::NeedSendAgain(""); - } // creating done - } else { // not auto partition - for (int i = 0; i < num_rows; ++i) { - if (UNLIKELY(has_filtered_rows) && _block_convertor->filter_bitmap().Get(i)) { - continue; - } - const VOlapTablePartition* partition = nullptr; - bool is_continue = false; - uint32_t tablet_index = 0; - RETURN_IF_ERROR(_tablet_finder->find_tablet(state, block.get(), i, &partition, - tablet_index, stop_processing, - is_continue)); - if (is_continue) { - continue; - } - // each row - _generate_row_distribution_payload(channel_to_payload, partition, tablet_index, i, - 1); + const VOlapTablePartition* partition = nullptr; + bool is_continue = false; + uint32_t tablet_index = 0; + RETURN_IF_ERROR(_tablet_finder->find_tablet( + state, block.get(), i, &partition, tablet_index, stop_processing, is_continue)); + if (is_continue) { + continue; } + // each row + _generate_row_distribution_payload(channel_to_payload, partition, tablet_index, i, 1); } } _row_distribution_watch.stop(); // Random distribution and the block belongs to a single tablet, we could optimize to append the whole // block into node channel. - bool load_block_to_single_tablet = - !_vpartition->is_auto_partition() && _tablet_finder->is_single_tablet(); + bool load_block_to_single_tablet = _tablet_finder->is_single_tablet(); if (load_block_to_single_tablet) { SCOPED_RAW_TIMER(&_filter_ns); // Filter block @@ -1629,17 +1372,12 @@ Status VOlapTableSink::send(RuntimeState* state, vectorized::Block* input_block, block.get(), filter_col, block->columns())); } } - // FIXME: Before load, we need to projection unuseful column - // auto slots = _schema->tuple_desc()->slots(); - // for (auto desc : slots) { - // desc->col_pos(); - // } // Add block to node channel for (size_t i = 0; i < _channels.size(); i++) { for (const auto& entry : channel_to_payload[i]) { // if this node channel is already failed, this add_row will be skipped auto st = entry.first->add_block( - block.get(), &entry.second, // entry.second is a [row -> tablet] mapping + block.get(), &entry.second, // if it is load single tablet, then append this whole block load_block_to_single_tablet); if (!st.ok()) { @@ -1754,7 +1492,6 @@ Status VOlapTableSink::close(RuntimeState* state, Status exec_status) { SCOPED_TIMER(_profile->total_time_counter()); try_close(state, exec_status); - // If _close_status is not ok, all nodes have been canceled in try_close. if (_close_status.ok()) { auto status = Status::OK(); @@ -1765,43 +1502,44 @@ Status VOlapTableSink::close(RuntimeState* state, Status exec_status) { total_wait_exec_time_ns = 0, max_wait_exec_time_ns = 0, total_add_batch_num = 0, num_node_channels = 0; VNodeChannelStat channel_stat; - - for (const auto& index_channel : _channels) { - if (!status.ok()) { - break; - } - int64_t add_batch_exec_time = 0; - int64_t wait_exec_time = 0; - index_channel->for_each_node_channel( - [this, &index_channel, &status, &state, &node_add_batch_counter_map, - &serialize_batch_ns, &channel_stat, &queue_push_lock_ns, &actual_consume_ns, - &total_add_batch_exec_time_ns, &add_batch_exec_time, &total_wait_exec_time_ns, - &wait_exec_time, - &total_add_batch_num](const std::shared_ptr& ch) { - if (!status.ok() || ch->is_closed()) { - return; - } - // in pipeline, all node channels are done or canceled, will not block. - // no pipeline, close may block waiting. - auto s = ch->close_wait(state); - if (!s.ok()) { - status = this->_cancel_channel_and_check_intolerable_failure( - status, s.to_string(), index_channel, ch); - } - ch->time_report(&node_add_batch_counter_map, &serialize_batch_ns, - &channel_stat, &queue_push_lock_ns, &actual_consume_ns, - &total_add_batch_exec_time_ns, &add_batch_exec_time, - &total_wait_exec_time_ns, &wait_exec_time, - &total_add_batch_num); - }); - num_node_channels += index_channel->num_node_channels(); - if (add_batch_exec_time > max_add_batch_exec_time_ns) { - max_add_batch_exec_time_ns = add_batch_exec_time; - } - if (wait_exec_time > max_wait_exec_time_ns) { - max_wait_exec_time_ns = wait_exec_time; - } - } // end for index channels + { + for (const auto& index_channel : _channels) { + if (!status.ok()) { + break; + } + int64_t add_batch_exec_time = 0; + int64_t wait_exec_time = 0; + index_channel->for_each_node_channel( + [this, &index_channel, &status, &state, &node_add_batch_counter_map, + &serialize_batch_ns, &channel_stat, &queue_push_lock_ns, + &actual_consume_ns, &total_add_batch_exec_time_ns, &add_batch_exec_time, + &total_wait_exec_time_ns, &wait_exec_time, + &total_add_batch_num](const std::shared_ptr& ch) { + if (!status.ok() || ch->is_closed()) { + return; + } + // in pipeline, all node channels are done or canceled, will not block. + // no pipeline, close may block waiting. + auto s = ch->close_wait(state); + if (!s.ok()) { + status = this->_cancel_channel_and_check_intolerable_failure( + status, s.to_string(), index_channel, ch); + } + ch->time_report(&node_add_batch_counter_map, &serialize_batch_ns, + &channel_stat, &queue_push_lock_ns, &actual_consume_ns, + &total_add_batch_exec_time_ns, &add_batch_exec_time, + &total_wait_exec_time_ns, &wait_exec_time, + &total_add_batch_num); + }); + num_node_channels += index_channel->num_node_channels(); + if (add_batch_exec_time > max_add_batch_exec_time_ns) { + max_add_batch_exec_time_ns = add_batch_exec_time; + } + if (wait_exec_time > max_wait_exec_time_ns) { + max_wait_exec_time_ns = wait_exec_time; + } + } // end for index channels + } if (status.ok()) { // TODO need to be improved diff --git a/be/src/vec/sink/vtablet_sink.h b/be/src/vec/sink/vtablet_sink.h index 7b37acf1f9eaa0e..dc408d392b24c97 100644 --- a/be/src/vec/sink/vtablet_sink.h +++ b/be/src/vec/sink/vtablet_sink.h @@ -20,9 +20,6 @@ #include #include #include -#include -#include -#include #include #include #include @@ -35,7 +32,6 @@ #include // IWYU pragma: no_include #include // IWYU pragma: keep -#include #include #include #include @@ -44,7 +40,6 @@ #include #include #include -#include #include #include #include @@ -159,8 +154,6 @@ class ReusableClosure final : public google::protobuf::Closure { cid = cntl.call_id(); } - // if _packet_in_flight == false, set it to true. Return true. - // if _packet_in_flight == true, Return false. bool try_set_in_flight() { bool value = false; return _packet_in_flight.compare_exchange_strong(value, true); @@ -218,47 +211,31 @@ class VNodeChannelStat { int64_t append_node_channel_ns = 0; }; -// every NodeChannel keeps a data transmission channel with one BE. for multiple times open, it has a dozen of requests and corresponding closures. class VNodeChannel { public: - VNodeChannel(VOlapTableSink* parent, IndexChannel* index_channel, int64_t node_id, - bool is_incremental = false); + VNodeChannel(VOlapTableSink* parent, IndexChannel* index_channel, int64_t node_id); ~VNodeChannel(); - // called before open, used to add tablet located in this backend. called by IndexChannel::init + // called before open, used to add tablet located in this backend void add_tablet(const TTabletWithPartition& tablet) { _all_tablets.emplace_back(tablet); } - std::string debug_tablets() const { - std::stringstream ss; - for (auto& tab : _all_tablets) { - tab.printTo(ss); - ss << '\n'; - } - return ss.str(); - } void add_slave_tablet_nodes(int64_t tablet_id, const std::vector& slave_nodes) { _slave_tablet_nodes[tablet_id] = slave_nodes; } - // build a request and build corresponding connect to BE. void open(); - // for auto partition, we use this to open more tablet. - void incremental_open(); Status init(RuntimeState* state); - // this will block until all request transmission which were opened or incremental opened finished. Status open_wait(); Status add_block(vectorized::Block* block, const Payload* payload, bool is_append = false); - // @return: unfinished running channels. - // @caller: VOlapTabletSink::_send_batch_process. it's a continual asynchronous process. int try_send_and_fetch_status(RuntimeState* state, std::unique_ptr& thread_pool_token); - // when there's pending block found by try_send_and_fetch_status(), we will awake a thread to send it. - void try_send_pending_block(RuntimeState* state); + + void try_send_block(RuntimeState* state); void clear_all_blocks(); @@ -322,18 +299,10 @@ class VNodeChannel { size_t get_pending_bytes() { return _pending_batches_bytes; } - bool is_incremental() const { return _is_incremental; } - protected: - // make a real open request for relative BE's load channel. - void _open_internal(bool is_incremental); - void _close_check(); void _cancel_with_msg(const std::string& msg); - void _add_block_success_callback(const PTabletWriterAddBlockResult& result, bool is_last_rpc); - void _add_block_failed_callback(bool is_last_rpc); - VOlapTableSink* _parent = nullptr; IndexChannel* _index_channel = nullptr; int64_t _node_id = -1; @@ -376,8 +345,7 @@ class VNodeChannel { std::atomic _pending_batches_num {0}; // reuse for vectorized std::shared_ptr _stub = nullptr; - // because we have incremantal open, we should keep one relative closure for one request. it's similarly for adding block. - std::vector*> _open_closures; + RefCountClosure* _open_closure = nullptr; std::vector _all_tablets; // map from tablet_id to node_id where slave replicas locate in @@ -405,7 +373,6 @@ class VNodeChannel { // rows number received per tablet, tablet_id -> rows_num std::vector> _tablets_received_rows; - // build a _cur_mutable_block and push into _pending_blocks. when not building, this block is empty. std::unique_ptr _cur_mutable_block; PTabletWriterAddBlockRequest _cur_add_block_request; @@ -413,8 +380,6 @@ class VNodeChannel { std::pair, PTabletWriterAddBlockRequest>; std::queue _pending_blocks; ReusableClosure* _add_block_closure = nullptr; - - bool _is_incremental; }; // Write block data to Olap Table. @@ -468,16 +433,6 @@ class VOlapTableSink final : public DataSink { void _cancel_all_channel(Status status); - std::pair _get_partition_function(); - - void _save_missing_values(vectorized::ColumnPtr col, vectorized::DataTypePtr value_type, - std::vector filter); - - // create partitions when need for auto-partition table using #_partitions_need_create. - Status _automatic_create_partition(); - - Status _incremental_open_node_channel(const std::vector& partitions); - std::shared_ptr _mem_tracker; ObjectPool* _pool; @@ -509,16 +464,11 @@ class VOlapTableSink final : public DataSink { std::unique_ptr _tablet_finder; // index_channel - std::mutex _stop_check_channel; std::vector> _channels; - std::unordered_map> _index_id_to_channel; bthread_t _sender_thread = 0; std::unique_ptr _send_batch_thread_pool_token; - // support only one partition column now - std::vector> _partitions_need_create; - std::unique_ptr _block_convertor; // Stats for this int64_t _send_data_ns = 0; @@ -536,7 +486,6 @@ class VOlapTableSink final : public DataSink { RuntimeProfile::Counter* _append_node_channel_timer = nullptr; RuntimeProfile::Counter* _filter_timer = nullptr; RuntimeProfile::Counter* _where_clause_timer = nullptr; - RuntimeProfile::Counter* _add_partition_request_timer = nullptr; RuntimeProfile::Counter* _wait_mem_limit_timer = nullptr; RuntimeProfile::Counter* _validate_data_timer = nullptr; RuntimeProfile::Counter* _open_timer = nullptr; diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java b/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java index e33ef112e2e02be..1ae98bedcf64f6a 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java @@ -68,11 +68,9 @@ public final class FeMetaVersion { public static final int VERSION_123 = 123; // For auto-increment column public static final int VERSION_124 = 124; - // For write/read auto create partition expr - public static final int VERSION_125 = 125; // note: when increment meta version, should assign the latest version to VERSION_CURRENT - public static final int VERSION_CURRENT = VERSION_125; + public static final int VERSION_CURRENT = VERSION_124; // all logs meta version should >= the minimum version, so that we could remove many if clause, for example // if (FE_METAVERSION < VERSION_94) ... diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index 3cd7f9fd3fe6c20..7b32fefac5d3fcf 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -3228,28 +3228,7 @@ opt_partition ::= | KW_PARTITION KW_BY KW_LIST LPAREN ident_list:columns RPAREN LPAREN opt_all_partition_desc_list:list RPAREN {: - RESULT = new ListPartitionDesc(columns, list); - :} - /* expr range partition */ - | KW_AUTO KW_PARTITION KW_BY KW_RANGE function_call_expr:fnExpr - LPAREN opt_all_partition_desc_list:list RPAREN - {: - ArrayList exprs = new ArrayList(); - exprs.add(fnExpr); - RESULT = RangePartitionDesc.createRangePartitionDesc(exprs, list); - :} - /* expr list partition */ - | KW_AUTO KW_PARTITION KW_BY KW_LIST LPAREN expr_list:exprs RPAREN - LPAREN opt_all_partition_desc_list:list RPAREN - {: - RESULT = ListPartitionDesc.createListPartitionDesc(exprs, list); - :} - | KW_AUTO KW_PARTITION KW_BY KW_LIST function_call_expr:fnExpr - LPAREN opt_all_partition_desc_list:list RPAREN - {: - ArrayList exprs = new ArrayList(); - exprs.add(fnExpr); - RESULT = ListPartitionDesc.createListPartitionDesc(exprs, list); + RESULT = new ListPartitionDesc(columns, list); :} ; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ListPartitionDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ListPartitionDesc.java index 0ca97ca9605dcdb..d0b6bebf052c94c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ListPartitionDesc.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ListPartitionDesc.java @@ -36,24 +36,6 @@ public ListPartitionDesc(List partitionColNames, List allPartitionDescs) throws AnalysisException { super(partitionColNames, allPartitionDescs); type = PartitionType.LIST; - this.isAutoCreatePartitions = false; - } - - public ListPartitionDesc(ArrayList exprs, List partitionColNames, - List allPartitionDescs) throws AnalysisException { - if (exprs != null) { - this.partitionExprs = exprs; - } - this.partitionColNames = partitionColNames; - this.singlePartitionDescs = handleAllPartitionDesc(allPartitionDescs); - this.type = PartitionType.LIST; - this.isAutoCreatePartitions = true; - } - - public static ListPartitionDesc createListPartitionDesc(ArrayList exprs, - List allPartitionDescs) throws AnalysisException { - List colNames = getColNamesFromExpr(exprs, true); - return new ListPartitionDesc(exprs, colNames, allPartitionDescs); } @Override @@ -118,8 +100,7 @@ public PartitionInfo toPartitionInfo(List schema, Map part } } - ListPartitionInfo listPartitionInfo = new ListPartitionInfo(this.isAutoCreatePartitions, this.partitionExprs, - partitionColumns); + ListPartitionInfo listPartitionInfo = new ListPartitionInfo(partitionColumns); for (SinglePartitionDesc desc : singlePartitionDescs) { long partitionId = partitionNameToId.get(desc.getPartitionName()); listPartitionInfo.handleNewSinglePartitionDesc(desc, partitionId, isTemp); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionDesc.java index 8c5ff8b0ef8cfcc..11cb795fd617e03 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionDesc.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionDesc.java @@ -27,14 +27,11 @@ import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.qe.ConnectContext; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.ImmutableSortedSet; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.commons.lang3.NotImplementedException; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Set; @@ -42,23 +39,14 @@ public class PartitionDesc { protected List partitionColNames; protected List singlePartitionDescs; - protected ArrayList partitionExprs; //eg: auto partition by range date_trunc(column, 'day') - protected boolean isAutoCreatePartitions; + protected PartitionType type; - public static final ImmutableSet RANGE_PARTITION_FUNCTIONS = new ImmutableSortedSet.Builder( - String.CASE_INSENSITIVE_ORDER).add("date_trunc").add("date_ceil").add("date_floor") - .build(); public PartitionDesc() {} public PartitionDesc(List partitionColNames, List allPartitionDescs) throws AnalysisException { this.partitionColNames = partitionColNames; - this.singlePartitionDescs = handleAllPartitionDesc(allPartitionDescs); - } - - public List handleAllPartitionDesc(List allPartitionDescs) - throws AnalysisException { boolean isMultiPartition = false; List tmpList = Lists.newArrayList(); if (allPartitionDescs != null) { @@ -77,7 +65,7 @@ public List handleAllPartitionDesc(List a throw new AnalysisException("multi partition column size except 1 but provided " + partitionColNames.size() + "."); } - return tmpList; + this.singlePartitionDescs = tmpList; } public List getSinglePartitionDescs() { @@ -97,62 +85,6 @@ public List getPartitionColNames() { return partitionColNames; } - // 1. partition by list (column) : now support one slotRef - // 2. partition by range(column/function(column)) : support slotRef and some - // special function eg: date_trunc, date_floor/ceil - public static List getColNamesFromExpr(ArrayList exprs, boolean isListPartition) - throws AnalysisException { - List colNames = new ArrayList<>(); - for (Expr expr : exprs) { - if ((expr instanceof FunctionCallExpr) && (isListPartition == false)) { - FunctionCallExpr functionCallExpr = (FunctionCallExpr) expr; - List paramsExpr = functionCallExpr.getParams().exprs(); - String name = functionCallExpr.getFnName().getFunction(); - if (RANGE_PARTITION_FUNCTIONS.contains(name)) { - for (Expr param : paramsExpr) { - if (param instanceof SlotRef) { - if (colNames.isEmpty()) { - colNames.add(((SlotRef) param).getColumnName()); - } else { - throw new AnalysisException( - "auto create partition only support one slotRef in function expr. " - + expr.toSql()); - } - } - } - } else { - throw new AnalysisException( - "auto create partition only support function call expr is date_trunc/date_floor/date_ceil. " - + expr.toSql()); - } - } else if (expr instanceof SlotRef) { - if (colNames.isEmpty()) { - colNames.add(((SlotRef) expr).getColumnName()); - } else { - throw new AnalysisException( - "auto create partition only support one slotRef in expr. " - + expr.toSql()); - } - } else { - if (!isListPartition) { - throw new AnalysisException( - "auto create partition only support slotRef and date_trunc/date_floor/date_ceil" - + "function in range partitions. " + expr.toSql()); - } else { - throw new AnalysisException( - "auto create partition only support slotRef in list partitions. " - + expr.toSql()); - } - } - } - if (colNames.isEmpty()) { - throw new AnalysisException( - "auto create partition have not find any partition columns. " - + exprs.get(0).toSql()); - } - return colNames; - } - public void analyze(List columnDefs, Map otherProperties) throws AnalysisException { if (partitionColNames == null || partitionColNames.isEmpty()) { throw new AnalysisException("No partition columns."); @@ -196,15 +128,6 @@ public void analyze(List columnDefs, Map otherPropert if (this instanceof ListPartitionDesc && columnDef.isAllowNull()) { throw new AnalysisException("The list partition column must be NOT NULL"); } - if (this instanceof RangePartitionDesc && partitionExprs != null) { - if (partitionExprs.get(0) instanceof FunctionCallExpr) { - if (!columnDef.getType().isDatetime() && !columnDef.getType().isDatetimeV2()) { - throw new AnalysisException( - "auto create partition function expr need datetime/datetimev2 type. " - + partitionExprs.get(0).toSql()); - } - } - } found = true; break; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionExprUtil.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionExprUtil.java deleted file mode 100644 index dadf74b27c06d88..000000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionExprUtil.java +++ /dev/null @@ -1,193 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.analysis; - -import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.PartitionInfo; -import org.apache.doris.catalog.PartitionType; -import org.apache.doris.catalog.Type; -import org.apache.doris.common.AnalysisException; -import org.apache.doris.thrift.TStringLiteral; - -import com.google.common.collect.Maps; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -public class PartitionExprUtil { - public static final String DATETIME_FORMATTER = "%04d-%02d-%02d %02d:%02d:%02d"; - public static final String DATE_FORMATTER = "%04d-%02d-%02d"; - public static final String DATETIME_NAME_FORMATTER = "%04d%02d%02d%02d%02d%02d"; - private static final Logger LOG = LogManager.getLogger(PartitionExprUtil.class); - private static final PartitionExprUtil partitionExprUtil = new PartitionExprUtil(); - - public static FunctionIntervalInfo getFunctionIntervalInfo(ArrayList partitionExprs, - PartitionType partitionType) throws AnalysisException { - if (partitionType != PartitionType.RANGE) { - return null; - } - if (partitionExprs.size() != 1) { - throw new AnalysisException("now only support one expr in range partition"); - } - - Expr e = partitionExprs.get(0); - if (!(e instanceof FunctionCallExpr)) { - throw new AnalysisException("now range partition only support FunctionCallExpr"); - } - FunctionCallExpr functionCallExpr = (FunctionCallExpr) e; - String fnName = functionCallExpr.getFnName().getFunction(); - String timeUnit; - int interval; - if ("date_trunc".equalsIgnoreCase(fnName)) { - List paramsExprs = functionCallExpr.getParams().exprs(); - if (paramsExprs.size() != 2) { - throw new AnalysisException("date_trunc params exprs size should be 2."); - } - Expr param = paramsExprs.get(1); - if (!(param instanceof StringLiteral)) { - throw new AnalysisException("date_trunc param of time unit is not string literal."); - } - timeUnit = ((StringLiteral) param).getStringValue().toLowerCase(); - interval = 1; - } else { - throw new AnalysisException("now range partition only support date_trunc."); - } - return partitionExprUtil.new FunctionIntervalInfo(timeUnit, interval); - } - - public static DateLiteral getRangeEnd(DateLiteral beginTime, FunctionIntervalInfo intervalInfo) - throws AnalysisException { - String timeUnit = intervalInfo.timeUnit; - int interval = intervalInfo.interval; - switch (timeUnit) { - case "year": - return beginTime.plusYears(interval); - case "month": - return beginTime.plusMonths(interval); - case "day": - return beginTime.plusDays(interval); - case "hour": - return beginTime.plusHours(interval); - case "minute": - return beginTime.plusMinutes(interval); - case "second": - return beginTime.plusSeconds(interval); - default: - break; - } - return null; - } - - public static Map getAddPartitionClauseFromPartitionValues(OlapTable olapTable, - ArrayList partitionValues, PartitionInfo partitionInfo) - throws AnalysisException { - Map result = Maps.newHashMap(); - ArrayList partitionExprs = partitionInfo.getPartitionExprs(); - PartitionType partitionType = partitionInfo.getType(); - List partiitonColumn = partitionInfo.getPartitionColumns(); - Type partitionColumnType = partiitonColumn.get(0).getType(); - FunctionIntervalInfo intervalInfo = getFunctionIntervalInfo(partitionExprs, partitionType); - Set filterPartitionValues = new HashSet(); - - for (TStringLiteral partitionValue : partitionValues) { - PartitionKeyDesc partitionKeyDesc = null; - String partitionName = "p"; - String value = partitionValue.value; - if (filterPartitionValues.contains(value)) { - continue; - } - filterPartitionValues.add(value); - if (partitionType == PartitionType.RANGE) { - String beginTime = value; - DateLiteral beginDateTime = new DateLiteral(beginTime, Type.DATETIMEV2); - partitionName += String.format(DATETIME_NAME_FORMATTER, - beginDateTime.getYear(), beginDateTime.getMonth(), beginDateTime.getDay(), - beginDateTime.getHour(), beginDateTime.getMinute(), beginDateTime.getSecond()); - DateLiteral endDateTime = getRangeEnd(beginDateTime, intervalInfo); - partitionKeyDesc = createPartitionKeyDescWithRange(beginDateTime, endDateTime, partitionColumnType); - } else if (partitionType == PartitionType.LIST) { - List> listValues = new ArrayList<>(); - // TODO: need to support any type - String pointValue = value; - PartitionValue lowerValue = new PartitionValue(pointValue); - listValues.add(Collections.singletonList(lowerValue)); - partitionKeyDesc = PartitionKeyDesc.createIn( - listValues); - partitionName += lowerValue.getStringValue(); - } else { - throw new AnalysisException("now only support range and list partition"); - } - - Map partitionProperties = Maps.newHashMap(); - DistributionDesc distributionDesc = olapTable.getDefaultDistributionInfo().toDistributionDesc(); - - SinglePartitionDesc singleRangePartitionDesc = new SinglePartitionDesc(true, partitionName, - partitionKeyDesc, partitionProperties); - - AddPartitionClause addPartitionClause = new AddPartitionClause(singleRangePartitionDesc, - distributionDesc, partitionProperties, false); - result.put(partitionName, addPartitionClause); - } - return result; - } - - public static PartitionKeyDesc createPartitionKeyDescWithRange(DateLiteral beginDateTime, - DateLiteral endDateTime, Type partitionColumnType) throws AnalysisException { - String beginTime; - String endTime; - // maybe need check the range in FE also, like getAddPartitionClause. - if (partitionColumnType.isDate() || partitionColumnType.isDateV2()) { - beginTime = String.format(DATE_FORMATTER, beginDateTime.getYear(), beginDateTime.getMonth(), - beginDateTime.getDay()); - endTime = String.format(DATE_FORMATTER, endDateTime.getYear(), endDateTime.getMonth(), - endDateTime.getDay()); - } else if (partitionColumnType.isDatetime() || partitionColumnType.isDatetimeV2()) { - beginTime = String.format(DATETIME_FORMATTER, - beginDateTime.getYear(), beginDateTime.getMonth(), beginDateTime.getDay(), - beginDateTime.getHour(), beginDateTime.getMinute(), beginDateTime.getSecond()); - endTime = String.format(DATETIME_FORMATTER, - endDateTime.getYear(), endDateTime.getMonth(), endDateTime.getDay(), - endDateTime.getHour(), endDateTime.getMinute(), endDateTime.getSecond()); - } else { - throw new AnalysisException( - "not support range partition with column type : " + partitionColumnType.toString()); - } - PartitionValue lowerValue = new PartitionValue(beginTime); - PartitionValue upperValue = new PartitionValue(endTime); - return PartitionKeyDesc.createFixed( - Collections.singletonList(lowerValue), - Collections.singletonList(upperValue)); - } - - public class FunctionIntervalInfo { - public String timeUnit; - public int interval; - - public FunctionIntervalInfo(String timeUnit, int interval) { - this.timeUnit = timeUnit; - this.interval = interval; - } - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/RangePartitionDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/RangePartitionDesc.java index 099e5b0b21d4bdf..cc18df6299e418d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/RangePartitionDesc.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/RangePartitionDesc.java @@ -35,22 +35,6 @@ public RangePartitionDesc(List partitionColNames, List allPartitionDescs) throws AnalysisException { super(partitionColNames, allPartitionDescs); type = org.apache.doris.catalog.PartitionType.RANGE; - this.isAutoCreatePartitions = false; - } - - public RangePartitionDesc(ArrayList exprs, List partitionColNames, - List allPartitionDescs) throws AnalysisException { - this.partitionExprs = exprs; - this.partitionColNames = partitionColNames; - this.singlePartitionDescs = handleAllPartitionDesc(allPartitionDescs); - this.type = org.apache.doris.catalog.PartitionType.RANGE; - this.isAutoCreatePartitions = true; - } - - public static RangePartitionDesc createRangePartitionDesc(ArrayList exprs, - List allPartitionDescs) throws AnalysisException { - List colNames = getColNamesFromExpr(exprs, false); - return new RangePartitionDesc(exprs, colNames, allPartitionDescs); } @Override @@ -132,8 +116,7 @@ public PartitionInfo toPartitionInfo(List schema, Map part * [ {10, 100, 1000}, {50, 500, MIN } ) * [ {50, 500, MIN }, {80, MIN, MIN } ) */ - RangePartitionInfo rangePartitionInfo = new RangePartitionInfo(this.isAutoCreatePartitions, this.partitionExprs, - partitionColumns); + RangePartitionInfo rangePartitionInfo = new RangePartitionInfo(partitionColumns); for (SinglePartitionDesc desc : singlePartitionDescs) { long partitionId = partitionNameToId.get(desc.getPartitionName()); rangePartitionInfo.handleNewSinglePartitionDesc(desc, partitionId, isTemp); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ListPartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ListPartitionInfo.java index d657dc7d9d439a1..c7a6b5e5a153da6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/ListPartitionInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ListPartitionInfo.java @@ -18,13 +18,11 @@ package org.apache.doris.catalog; import org.apache.doris.analysis.AllPartitionDesc; -import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.ListPartitionDesc; import org.apache.doris.analysis.PartitionDesc; import org.apache.doris.analysis.PartitionKeyDesc; import org.apache.doris.analysis.PartitionValue; import org.apache.doris.analysis.SinglePartitionDesc; -import org.apache.doris.analysis.SlotRef; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; import org.apache.doris.common.util.ListUtil; @@ -56,14 +54,6 @@ public ListPartitionInfo(List partitionColumns) { this.isMultiColumnPartition = partitionColumns.size() > 1; } - public ListPartitionInfo(boolean isAutoCreatePartitions, ArrayList exprs, List partitionColumns) { - super(PartitionType.LIST, partitionColumns); - this.isAutoCreatePartitions = isAutoCreatePartitions; - if (exprs != null) { - this.partitionExprs.addAll(exprs); - } - } - public static PartitionInfo read(DataInput in) throws IOException { PartitionInfo partitionInfo = new ListPartitionInfo(); partitionInfo.readFields(in); @@ -196,31 +186,16 @@ public static void checkPartitionColumn(Column column) throws AnalysisException @Override public String toSql(OlapTable table, List partitionId) { StringBuilder sb = new StringBuilder(); + sb.append("PARTITION BY LIST("); int idx = 0; - if (enableAutomaticPartition()) { - sb.append("AUTO PARTITION BY LIST "); - for (Expr e : partitionExprs) { - boolean isSlotRef = (e instanceof SlotRef); - if (isSlotRef) { - sb.append("("); - } - sb.append(e.toSql()); - if (isSlotRef) { - sb.append(")"); - } - } - sb.append("\n("); - } else { - sb.append("PARTITION BY LIST("); - for (Column column : partitionColumns) { - if (idx != 0) { - sb.append(", "); - } - sb.append("`").append(column.getName()).append("`"); - idx++; + for (Column column : partitionColumns) { + if (idx != 0) { + sb.append(", "); } - sb.append(")\n("); + sb.append("`").append(column.getName()).append("`"); + idx++; } + sb.append(")\n("); // sort list List> entries = new ArrayList<>(this.idToItem.entrySet()); @@ -294,6 +269,6 @@ public PartitionDesc toPartitionDesc(OlapTable table) throws AnalysisException { allPartitionDescs.add(new SinglePartitionDesc(false, partitionName, partitionKeyDesc, properties)); } - return new ListPartitionDesc(this.partitionExprs, partitionColumnNames, allPartitionDescs); + return new ListPartitionDesc(partitionColumnNames, allPartitionDescs); } } 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 0d3ac83f7253ac3..b94440225b27f22 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 @@ -137,7 +137,7 @@ public enum OlapTableState { private PartitionInfo partitionInfo; @SerializedName("idToPartition") private Map idToPartition = new HashMap<>(); - private Map nameToPartition = Maps.newTreeMap(); + private Map nameToPartition = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); @SerializedName(value = "distributionInfo") private DistributionInfo defaultDistributionInfo; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java index b7ca3c622cc7b04..d319882af43f033 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java @@ -18,7 +18,6 @@ package org.apache.doris.catalog; import org.apache.doris.analysis.DateLiteral; -import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.MaxLiteral; import org.apache.doris.analysis.PartitionDesc; import org.apache.doris.analysis.PartitionValue; @@ -41,7 +40,6 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -83,10 +81,6 @@ public class PartitionInfo implements Writable { // so we defer adding meta serialization until memory engine feature is more complete. protected Map idToTabletType; - // the enable automatic partition will hold this, could create partition by expr result - protected ArrayList partitionExprs; - protected boolean isAutoCreatePartitions; - public PartitionInfo() { this.type = PartitionType.UNPARTITIONED; this.idToDataProperty = new HashMap<>(); @@ -94,7 +88,6 @@ public PartitionInfo() { this.idToInMemory = new HashMap<>(); this.idToTabletType = new HashMap<>(); this.idToStoragePolicy = new HashMap<>(); - this.partitionExprs = new ArrayList<>(); } public PartitionInfo(PartitionType type) { @@ -104,7 +97,6 @@ public PartitionInfo(PartitionType type) { this.idToInMemory = new HashMap<>(); this.idToTabletType = new HashMap<>(); this.idToStoragePolicy = new HashMap<>(); - this.partitionExprs = new ArrayList<>(); } public PartitionInfo(PartitionType type, List partitionColumns) { @@ -223,14 +215,6 @@ public PartitionItem getAnyIntersectItem(PartitionItem newItem, boolean isTemp) return null; } - public boolean enableAutomaticPartition() { - return isAutoCreatePartitions; - } - - public ArrayList getPartitionExprs() { - return this.partitionExprs; - } - public void checkPartitionItemListsMatch(List list1, List list2) throws DdlException { } @@ -390,13 +374,6 @@ public void write(DataOutput out) throws IOException { idToReplicaAllocation.get(entry.getKey()).write(out); out.writeBoolean(idToInMemory.get(entry.getKey())); } - int size = partitionExprs.size(); - out.writeInt(size); - for (int i = 0; i < size; ++i) { - Expr e = this.partitionExprs.get(i); - Expr.writeTo(e, out); - } - out.writeBoolean(isAutoCreatePartitions); } public void readFields(DataInput in) throws IOException { @@ -423,14 +400,6 @@ public void readFields(DataInput in) throws IOException { idToInMemory.put(partitionId, in.readBoolean()); } - if (Env.getCurrentEnvJournalVersion() >= FeMetaVersion.VERSION_125) { - int size = in.readInt(); - for (int i = 0; i < size; ++i) { - Expr e = Expr.readIn(in); - this.partitionExprs.add(e); - } - this.isAutoCreatePartitions = in.readBoolean(); - } } @Override @@ -469,13 +438,12 @@ public boolean equals(Object o) { && Objects.equals(idToTempItem, that.idToTempItem) && Objects.equals(idToDataProperty, that.idToDataProperty) && Objects.equals(idToStoragePolicy, that.idToStoragePolicy) && Objects.equals(idToReplicaAllocation, that.idToReplicaAllocation) && Objects.equals( - idToInMemory, that.idToInMemory) && Objects.equals(idToTabletType, that.idToTabletType) - && Objects.equals(partitionExprs, that.partitionExprs); + idToInMemory, that.idToInMemory) && Objects.equals(idToTabletType, that.idToTabletType); } @Override public int hashCode() { return Objects.hash(type, partitionColumns, idToItem, idToTempItem, idToDataProperty, idToStoragePolicy, - idToReplicaAllocation, isMultiColumnPartition, idToInMemory, idToTabletType, partitionExprs); + idToReplicaAllocation, isMultiColumnPartition, idToInMemory, idToTabletType); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionInfo.java index 952fa88d2592927..cd3614bec8bef35 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionInfo.java @@ -18,12 +18,10 @@ package org.apache.doris.catalog; import org.apache.doris.analysis.AllPartitionDesc; -import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.PartitionDesc; import org.apache.doris.analysis.PartitionKeyDesc; import org.apache.doris.analysis.RangePartitionDesc; import org.apache.doris.analysis.SinglePartitionDesc; -import org.apache.doris.analysis.SlotRef; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; import org.apache.doris.common.util.RangeUtils; @@ -56,14 +54,6 @@ public RangePartitionInfo(List partitionColumns) { this.isMultiColumnPartition = partitionColumns.size() > 1; } - public RangePartitionInfo(boolean isAutoCreatePartitions, ArrayList exprs, List partitionColumns) { - super(PartitionType.RANGE, partitionColumns); - this.isAutoCreatePartitions = isAutoCreatePartitions; - if (exprs != null) { - this.partitionExprs.addAll(exprs); - } - } - @Override public PartitionItem createAndCheckPartitionItem(SinglePartitionDesc desc, boolean isTemp) throws DdlException { Range newRange = null; @@ -262,31 +252,16 @@ public void readFields(DataInput in) throws IOException { @Override public String toSql(OlapTable table, List partitionId) { StringBuilder sb = new StringBuilder(); + sb.append("PARTITION BY RANGE("); int idx = 0; - if (enableAutomaticPartition()) { - sb.append("AUTO PARTITION BY RANGE "); - for (Expr e : partitionExprs) { - boolean isSlotRef = (e instanceof SlotRef); - if (isSlotRef) { - sb.append("("); - } - sb.append(e.toSql()); - if (isSlotRef) { - sb.append(")"); - } - } - sb.append("\n("); - } else { - sb.append("PARTITION BY RANGE("); - for (Column column : partitionColumns) { - if (idx != 0) { - sb.append(", "); - } - sb.append("`").append(column.getName()).append("`"); - idx++; + for (Column column : partitionColumns) { + if (idx != 0) { + sb.append(", "); } - sb.append(")\n("); + sb.append("`").append(column.getName()).append("`"); + idx++; } + sb.append(")\n("); // sort range List> entries = new ArrayList<>(this.idToItem.entrySet()); @@ -350,6 +325,6 @@ public PartitionDesc toPartitionDesc(OlapTable table) throws AnalysisException { allPartitionDescs.add(new SinglePartitionDesc(false, partitionName, partitionKeyDesc, properties)); } - return new RangePartitionDesc(this.partitionExprs, partitionColumnNames, allPartitionDescs); + return new RangePartitionDesc(partitionColumnNames, allPartitionDescs); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapTableSink.java index c238dcc2627173d..24e3cf2029cc2e1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapTableSink.java @@ -131,7 +131,7 @@ public void init(TUniqueId loadId, long txnId, long dbId, long loadChannelTimeou if (partitionIds == null) { partitionIds = dstTable.getPartitionIds(); - if (partitionIds.isEmpty() && dstTable.getPartitionInfo().enableAutomaticPartition() == false) { + if (partitionIds.isEmpty()) { ErrorReport.reportAnalysisException(ErrorCode.ERR_EMPTY_PARTITION_IN_TABLE, dstTable.getName()); } } @@ -178,7 +178,7 @@ public void complete(Analyzer analyzer) throws UserException { tSink.setNumReplicas(numReplicas); tSink.setNeedGenRollup(dstTable.shouldLoadToNewRollup()); tSink.setSchema(createSchema(tSink.getDbId(), dstTable, analyzer)); - tSink.setPartition(createPartition(tSink.getDbId(), dstTable, analyzer)); + tSink.setPartition(createPartition(tSink.getDbId(), dstTable)); List locationParams = createLocation(dstTable); tSink.setLocation(locationParams.get(0)); if (singleReplicaLoad) { @@ -293,8 +293,7 @@ private List getDistColumns(DistributionInfo distInfo) throws UserExcept return distColumns; } - private TOlapTablePartitionParam createPartition(long dbId, OlapTable table, Analyzer analyzer) - throws UserException { + private TOlapTablePartitionParam createPartition(long dbId, OlapTable table) throws UserException { TOlapTablePartitionParam partitionParam = new TOlapTablePartitionParam(); partitionParam.setDbId(dbId); partitionParam.setTableId(table.getId()); @@ -338,22 +337,6 @@ private TOlapTablePartitionParam createPartition(long dbId, OlapTable table, Ana } } } - // for auto create partition by function expr, there is no any partition firstly, - // But this is required in thrift struct. - if (partitionIds.isEmpty()) { - partitionParam.setDistributedColumns(getDistColumns(table.getDefaultDistributionInfo())); - partitionParam.setPartitions(new ArrayList()); - } - ArrayList exprs = partitionInfo.getPartitionExprs(); - if (exprs != null && analyzer != null) { - tupleDescriptor.setTable(table); - analyzer.registerTupleDescriptor(tupleDescriptor); - for (Expr e : exprs) { - e.analyze(analyzer); - } - partitionParam.setPartitionFunctionExprs(Expr.treesToThrift(exprs)); - } - partitionParam.setEnableAutomaticPartition(partitionInfo.enableAutomaticPartition()); break; } case UNPARTITIONED: { @@ -379,18 +362,16 @@ private TOlapTablePartitionParam createPartition(long dbId, OlapTable table, Ana } partitionParam.addToPartitions(tPartition); partitionParam.setDistributedColumns(getDistColumns(partition.getDistributionInfo())); - partitionParam.setEnableAutomaticPartition(false); break; } default: { throw new UserException("unsupported partition for OlapTable, partition=" + partType); } } - partitionParam.setPartitionType(partType.toThrift()); return partitionParam; } - public static void setPartitionKeys(TOlapTablePartition tPartition, PartitionItem partitionItem, int partColNum) { + private void setPartitionKeys(TOlapTablePartition tPartition, PartitionItem partitionItem, int partColNum) { if (partitionItem instanceof RangePartitionItem) { Range range = partitionItem.getItems(); // set start keys @@ -458,10 +439,6 @@ private List createLocation(OlapTable table) throws Use } } - // for partition by function expr, there is no any partition firstly, But this is required in thrift struct. - if (partitionIds.isEmpty()) { - locationParam.setTablets(new ArrayList()); - } // check if disk capacity reach limit // this is for load process, so use high water mark to check Status st = Env.getCurrentSystemInfo().checkExceedDiskCapacityLimit(allBePathsMap, true); diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index d22ae9b7f26f43c..af7a04b45f54b17 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -19,12 +19,10 @@ import org.apache.doris.alter.SchemaChangeHandler; import org.apache.doris.analysis.AddColumnsClause; -import org.apache.doris.analysis.AddPartitionClause; import org.apache.doris.analysis.Analyzer; import org.apache.doris.analysis.ColumnDef; import org.apache.doris.analysis.LabelName; import org.apache.doris.analysis.NativeInsertStmt; -import org.apache.doris.analysis.PartitionExprUtil; import org.apache.doris.analysis.RestoreStmt; import org.apache.doris.analysis.SetType; import org.apache.doris.analysis.SqlParser; @@ -39,10 +37,7 @@ import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.Index; -import org.apache.doris.catalog.MaterializedIndex; import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.Partition; -import org.apache.doris.catalog.PartitionInfo; import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.TableIf; @@ -55,7 +50,6 @@ import org.apache.doris.common.AuthenticationException; import org.apache.doris.common.CaseSensibility; import org.apache.doris.common.Config; -import org.apache.doris.common.DdlException; import org.apache.doris.common.DuplicatedRequestException; import org.apache.doris.common.LabelAlreadyUsedException; import org.apache.doris.common.MetaNotFoundException; @@ -78,7 +72,6 @@ import org.apache.doris.mysql.privilege.AccessControllerManager; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.persist.gson.GsonUtils; -import org.apache.doris.planner.OlapTableSink; import org.apache.doris.planner.StreamLoadPlanner; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ConnectProcessor; @@ -117,8 +110,6 @@ import org.apache.doris.thrift.TCommitTxnResult; import org.apache.doris.thrift.TConfirmUnusedRemoteFilesRequest; import org.apache.doris.thrift.TConfirmUnusedRemoteFilesResult; -import org.apache.doris.thrift.TCreatePartitionRequest; -import org.apache.doris.thrift.TCreatePartitionResult; import org.apache.doris.thrift.TDescribeTableParams; import org.apache.doris.thrift.TDescribeTableResult; import org.apache.doris.thrift.TDescribeTablesParams; @@ -164,9 +155,6 @@ import org.apache.doris.thrift.TMasterResult; import org.apache.doris.thrift.TMySqlLoadAcquireTokenResult; import org.apache.doris.thrift.TNetworkAddress; -import org.apache.doris.thrift.TNodeInfo; -import org.apache.doris.thrift.TOlapTableIndexTablets; -import org.apache.doris.thrift.TOlapTablePartition; import org.apache.doris.thrift.TPipelineFragmentParams; import org.apache.doris.thrift.TPrivilegeCtrl; import org.apache.doris.thrift.TPrivilegeHier; @@ -192,12 +180,10 @@ import org.apache.doris.thrift.TStreamLoadMultiTablePutResult; import org.apache.doris.thrift.TStreamLoadPutRequest; import org.apache.doris.thrift.TStreamLoadPutResult; -import org.apache.doris.thrift.TStringLiteral; import org.apache.doris.thrift.TTableIndexQueryStats; import org.apache.doris.thrift.TTableMetadataNameIds; import org.apache.doris.thrift.TTableQueryStats; import org.apache.doris.thrift.TTableStatus; -import org.apache.doris.thrift.TTabletLocation; import org.apache.doris.thrift.TTxnParams; import org.apache.doris.thrift.TUpdateExportTaskStatusRequest; import org.apache.doris.thrift.TUpdateFollowerStatsCacheRequest; @@ -214,7 +200,6 @@ import com.google.common.base.Strings; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.collect.Multimap; import org.apache.commons.collections.CollectionUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -3042,124 +3027,4 @@ public TStatus updateStatsCache(TUpdateFollowerStatsCacheRequest request) throws // Return Ok anyway return new TStatus(TStatusCode.OK); } - - @Override - public TCreatePartitionResult createPartition(TCreatePartitionRequest request) throws TException { - LOG.info("Receive create partition request: {}", request); - long dbId = request.getDbId(); - long tableId = request.getTableId(); - TCreatePartitionResult result = new TCreatePartitionResult(); - TStatus errorStatus = new TStatus(TStatusCode.RUNTIME_ERROR); - - Database db = Env.getCurrentEnv().getInternalCatalog().getDbNullable(dbId); - if (db == null) { - errorStatus.setErrorMsgs(Lists.newArrayList(String.format("dbId=%d is not exists", dbId))); - result.setStatus(errorStatus); - return result; - } - - Table table = db.getTable(tableId).get(); - if (table == null) { - errorStatus.setErrorMsgs( - (Lists.newArrayList(String.format("dbId=%d tableId=%d is not exists", dbId, tableId)))); - result.setStatus(errorStatus); - return result; - } - - if (!(table instanceof OlapTable)) { - errorStatus.setErrorMsgs( - Lists.newArrayList(String.format("dbId=%d tableId=%d is not olap table", dbId, tableId))); - result.setStatus(errorStatus); - return result; - } - - if (request.partitionValues == null) { - errorStatus.setErrorMsgs(Lists.newArrayList("partitionValues should not null.")); - result.setStatus(errorStatus); - return result; - } - - OlapTable olapTable = (OlapTable) table; - PartitionInfo partitionInfo = olapTable.getPartitionInfo(); - ArrayList partitionValues = new ArrayList(); - for (int i = 0; i < request.partitionValues.size(); i++) { - if (request.partitionValues.get(i).size() != 1) { - errorStatus.setErrorMsgs( - Lists.newArrayList("Only support single partition, partitionValues size should equal 1.")); - result.setStatus(errorStatus); - return result; - } - partitionValues.add(request.partitionValues.get(i).get(0)); - } - Map addPartitionClauseMap; - try { - addPartitionClauseMap = PartitionExprUtil.getAddPartitionClauseFromPartitionValues(olapTable, - partitionValues, partitionInfo); - } catch (AnalysisException ex) { - errorStatus.setErrorMsgs(Lists.newArrayList(ex.getMessage())); - result.setStatus(errorStatus); - return result; - } - - for (AddPartitionClause addPartitionClause : addPartitionClauseMap.values()) { - try { - // here maybe check and limit created partitions num - Env.getCurrentEnv().addPartition(db, olapTable.getName(), addPartitionClause); - } catch (DdlException e) { - LOG.warn(e); - errorStatus.setErrorMsgs( - Lists.newArrayList(String.format("create partition failed. error:%s", e.getMessage()))); - result.setStatus(errorStatus); - return result; - } - } - - // build partition & tablets - List partitions = Lists.newArrayList(); - List tablets = Lists.newArrayList(); - for (String partitionName : addPartitionClauseMap.keySet()) { - Partition partition = table.getPartition(partitionName); - TOlapTablePartition tPartition = new TOlapTablePartition(); - tPartition.setId(partition.getId()); - int partColNum = partitionInfo.getPartitionColumns().size(); - // set partition keys - OlapTableSink.setPartitionKeys(tPartition, partitionInfo.getItem(partition.getId()), partColNum); - for (MaterializedIndex index : partition.getMaterializedIndices(MaterializedIndex.IndexExtState.ALL)) { - tPartition.addToIndexes(new TOlapTableIndexTablets(index.getId(), Lists.newArrayList( - index.getTablets().stream().map(Tablet::getId).collect(Collectors.toList())))); - tPartition.setNumBuckets(index.getTablets().size()); - } - tPartition.setIsMutable(olapTable.getPartitionInfo().getIsMutable(partition.getId())); - partitions.add(tPartition); - // tablet - int quorum = olapTable.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum() / 2 - + 1; - for (MaterializedIndex index : partition.getMaterializedIndices(MaterializedIndex.IndexExtState.ALL)) { - for (Tablet tablet : index.getTablets()) { - // we should ensure the replica backend is alive - // otherwise, there will be a 'unknown node id, id=xxx' error for stream load - // BE id -> path hash - Multimap bePathsMap = tablet.getNormalReplicaBackendPathMap(); - if (bePathsMap.keySet().size() < quorum) { - LOG.warn("auto go quorum exception"); - } - tablets.add(new TTabletLocation(tablet.getId(), Lists.newArrayList(bePathsMap.keySet()))); - } - } - } - result.setPartitions(partitions); - result.setTablets(tablets); - - // build nodes - List nodeInfos = Lists.newArrayList(); - SystemInfoService systemInfoService = Env.getCurrentSystemInfo(); - for (Long id : systemInfoService.getAllBackendIds(false)) { - Backend backend = systemInfoService.getBackend(id); - nodeInfos.add(new TNodeInfo(backend.getId(), 0, backend.getHost(), backend.getBrpcPort())); - } - result.setNodes(nodeInfos); - result.setStatus(new TStatus(TStatusCode.OK)); - LOG.debug("send create partition result: {}", result); - return result; - } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/TruncateTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/TruncateTableTest.java index c13730f32d32790..0a05d7c618a205a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/TruncateTableTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/TruncateTableTest.java @@ -84,27 +84,26 @@ public static void tearDown() { @Test public void testTruncateWithCaseInsensitivePartitionName() throws Exception { - //now in order to support auto create partition, need set partition name is case sensitive Database db = Env.getCurrentInternalCatalog().getDbNullable("default_cluster:test"); OlapTable tbl = db.getOlapTableOrDdlException("case_sensitive_table"); - long p20211006Id = tbl.getPartition("p20211006").getId(); + long p20211006Id = tbl.getPartition("P20211006").getId(); long p20211007Id = tbl.getPartition("P20211007").getId(); - long p20211008Id = tbl.getPartition("P20211008").getId(); + long p20211008Id = tbl.getPartition("p20211008").getId(); // truncate p20211008(real name is P20211008) - String truncateStr = "TRUNCATE TABLE test.case_sensitive_table PARTITION P20211008; \n"; + String truncateStr = "TRUNCATE TABLE test.case_sensitive_table PARTITION p20211008; \n"; TruncateTableStmt truncateTableStmt = (TruncateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(truncateStr, connectContext); Env.getCurrentEnv().truncateTable(truncateTableStmt); - Assert.assertNotEquals(p20211008Id, tbl.getPartition("P20211008").getId()); + Assert.assertNotEquals(p20211008Id, tbl.getPartition("p20211008").getId()); // 2. truncate P20211007 truncateStr = "TRUNCATE TABLE test.case_sensitive_table PARTITION P20211007; \n"; truncateTableStmt = (TruncateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(truncateStr, connectContext); Env.getCurrentEnv().truncateTable(truncateTableStmt); Assert.assertEquals(3, tbl.getPartitionInfo().idToDataProperty.size()); - Assert.assertNotEquals(p20211007Id, tbl.getPartition("P20211007").getId()); + Assert.assertNotEquals(p20211007Id, tbl.getPartition("p20211007").getId()); Assert.assertEquals(p20211006Id, tbl.getPartition("p20211006").getId()); Assert.assertNotNull(tbl.getPartition("p20211006")); - Assert.assertNotNull(tbl.getPartition("p20211006")); + Assert.assertNotNull(tbl.getPartition("P20211006")); } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/common/util/DynamicPartitionUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/common/util/DynamicPartitionUtilTest.java index 18ebaf6851aa281..ac9aded5b15d92e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/common/util/DynamicPartitionUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/common/util/DynamicPartitionUtilTest.java @@ -222,7 +222,7 @@ public void testCheckTimeUnit() { List partitionColumnList = Lists.newArrayList(); Column partitionColumn = new Column(); partitionColumn.setType(Type.DATE); - Deencapsulation.setField(rangePartitionInfo, "partitionColumns", partitionColumnList); + Deencapsulation.setField(rangePartitionInfo, partitionColumnList); try { Deencapsulation.invoke(dynamicPartitionUtil, "checkTimeUnit", "HOUR", rangePartitionInfo); Assert.fail(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/service/FrontendServiceImplTest.java b/fe/fe-core/src/test/java/org/apache/doris/service/FrontendServiceImplTest.java deleted file mode 100644 index 9a1264776af2ceb..000000000000000 --- a/fe/fe-core/src/test/java/org/apache/doris/service/FrontendServiceImplTest.java +++ /dev/null @@ -1,159 +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.service; - - -import org.apache.doris.analysis.CreateDbStmt; -import org.apache.doris.analysis.CreateTableStmt; -import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.Env; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.Partition; -import org.apache.doris.common.Config; -import org.apache.doris.common.FeConstants; -import org.apache.doris.qe.ConnectContext; -import org.apache.doris.thrift.TCreatePartitionRequest; -import org.apache.doris.thrift.TCreatePartitionResult; -import org.apache.doris.thrift.TStatusCode; -import org.apache.doris.thrift.TStringLiteral; -import org.apache.doris.utframe.UtFrameUtils; - -import mockit.Mocked; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; - -import java.util.ArrayList; -import java.util.List; -import java.util.UUID; - -public class FrontendServiceImplTest { - private static String runningDir = "fe/mocked/FrontendServiceImplTest/" + UUID.randomUUID().toString() + "/"; - private static ConnectContext connectContext; - @Rule - public ExpectedException expectedException = ExpectedException.none(); - @Mocked - ExecuteEnv exeEnv; - - @BeforeClass - public static void beforeClass() throws Exception { - FeConstants.runningUnitTest = true; - FeConstants.default_scheduler_interval_millisecond = 100; - Config.dynamic_partition_enable = true; - Config.dynamic_partition_check_interval_seconds = 1; - UtFrameUtils.createDorisCluster(runningDir); - // create connect context - connectContext = UtFrameUtils.createDefaultCtx(); - // create database - String createDbStmtStr = "create database test;"; - CreateDbStmt createDbStmt = (CreateDbStmt) UtFrameUtils.parseAndAnalyzeStmt(createDbStmtStr, connectContext); - Env.getCurrentEnv().createDb(createDbStmt); - } - - @AfterClass - public static void tearDown() { - UtFrameUtils.cleanDorisFeDir(runningDir); - } - - private static void createTable(String sql) throws Exception { - CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(sql, connectContext); - Env.getCurrentEnv().createTable(createTableStmt); - } - - - @Test - public void testCreatePartitionRange() throws Exception { - String createOlapTblStmt = new String("CREATE TABLE test.partition_range(\n" - + " event_day DATETIME,\n" - + " site_id INT DEFAULT '10',\n" - + " city_code VARCHAR(100)\n" - + ")\n" - + "DUPLICATE KEY(event_day, site_id, city_code)\n" - + "AUTO PARTITION BY range date_trunc( event_day,'day') (\n" - + "\n" - + ")\n" - + "DISTRIBUTED BY HASH(event_day, site_id) BUCKETS 2\n" - + "PROPERTIES(\"replication_num\" = \"1\");"); - - createTable(createOlapTblStmt); - Database db = Env.getCurrentInternalCatalog().getDbOrAnalysisException("default_cluster:test"); - OlapTable table = (OlapTable) db.getTableOrAnalysisException("partition_range"); - - List> partitionValues = new ArrayList<>(); - List values = new ArrayList<>(); - - TStringLiteral start = new TStringLiteral(); - start.setValue("2023-08-07 00:00:00"); - values.add(start); - - partitionValues.add(values); - - FrontendServiceImpl impl = new FrontendServiceImpl(exeEnv); - TCreatePartitionRequest request = new TCreatePartitionRequest(); - request.setDbId(db.getId()); - request.setTableId(table.getId()); - request.setPartitionValues(partitionValues); - TCreatePartitionResult partition = impl.createPartition(request); - - Assert.assertEquals(partition.getStatus().getStatusCode(), TStatusCode.OK); - Partition p20230807 = table.getPartition("p20230807000000"); - Assert.assertNotNull(p20230807); - } - - @Test - public void testCreatePartitionList() throws Exception { - String createOlapTblStmt = new String("CREATE TABLE test.partition_list(\n" - + " event_day DATETIME,\n" - + " site_id INT DEFAULT '10',\n" - + " city_code VARCHAR(100) not null\n" - + ")\n" - + "DUPLICATE KEY(event_day, site_id, city_code)\n" - + "AUTO PARTITION BY list (city_code) (\n" - + "\n" - + ")\n" - + "DISTRIBUTED BY HASH(event_day, site_id) BUCKETS 2\n" - + "PROPERTIES(\"replication_num\" = \"1\");"); - - createTable(createOlapTblStmt); - Database db = Env.getCurrentInternalCatalog().getDbOrAnalysisException("default_cluster:test"); - OlapTable table = (OlapTable) db.getTableOrAnalysisException("partition_list"); - - List> partitionValues = new ArrayList<>(); - List values = new ArrayList<>(); - - TStringLiteral start = new TStringLiteral(); - start.setValue("BEIJING"); - values.add(start); - - partitionValues.add(values); - - FrontendServiceImpl impl = new FrontendServiceImpl(exeEnv); - TCreatePartitionRequest request = new TCreatePartitionRequest(); - request.setDbId(db.getId()); - request.setTableId(table.getId()); - request.setPartitionValues(partitionValues); - TCreatePartitionResult partition = impl.createPartition(request); - - Assert.assertEquals(partition.getStatus().getStatusCode(), TStatusCode.OK); - Partition pbeijing = table.getPartition("pBEIJING"); - Assert.assertNotNull(pbeijing); - } -} diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto index 11a1d747bb1dee2..5a462987d2e5311 100644 --- a/gensrc/proto/internal_service.proto +++ b/gensrc/proto/internal_service.proto @@ -94,7 +94,6 @@ message PTabletWriterOpenRequest { optional bool is_vectorized = 12 [default = false]; optional int64 backend_id = 13 [default = -1]; optional bool enable_profile = 14 [default = false]; - optional bool is_incremental = 15 [default = false]; }; message PTabletWriterOpenResult { diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift index 1b9f85e2b7da487..302057087685449 100644 --- a/gensrc/thrift/Descriptors.thrift +++ b/gensrc/thrift/Descriptors.thrift @@ -20,7 +20,6 @@ namespace java org.apache.doris.thrift include "Types.thrift" include "Exprs.thrift" -include "Partitions.thrift" struct TColumn { 1: required string column_name @@ -192,9 +191,6 @@ struct TOlapTablePartitionParam { 6: required list partitions 7: optional list partition_columns - 8: optional list partition_function_exprs - 9: optional bool enable_automatic_partition - 10: optional Partitions.TPartitionType partition_type } struct TOlapTableIndex { diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index e6215cb1cadd099..c0ef12dadb6ec65 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -1130,21 +1130,6 @@ struct TAutoIncrementRangeResult { 3: optional i64 length } -struct TCreatePartitionRequest { - 1: optional i64 txn_id - 2: optional i64 db_id - 3: optional i64 table_id - // for each partition column's partition values. [missing_rows, partition_keys]->Left bound(for range) or Point(for list) - 4: optional list> partitionValues -} - -struct TCreatePartitionResult { - 1: optional Status.TStatus status - 2: optional list partitions - 3: optional list tablets - 4: optional list nodes -} - service FrontendService { TGetDbsResult getDbNames(1: TGetDbsParams params) TGetTablesResult getTableNames(1: TGetTablesParams params) @@ -1214,6 +1199,4 @@ service FrontendService { Status.TStatus updateStatsCache(1: TUpdateFollowerStatsCacheRequest request) TAutoIncrementRangeResult getAutoIncrementRange(1: TAutoIncrementRangeRequest request) - - TCreatePartitionResult createPartition(1: TCreatePartitionRequest request) } diff --git a/regression-test/data/partition_p0/auto_partition/auto_partition_stream_load1.csv b/regression-test/data/partition_p0/auto_partition/auto_partition_stream_load1.csv deleted file mode 100644 index ada7d3c6af6b0a4..000000000000000 --- a/regression-test/data/partition_p0/auto_partition/auto_partition_stream_load1.csv +++ /dev/null @@ -1,10 +0,0 @@ -1,2001-12-12 12:12:12.123,2001-12-12 12:12:12.123456 -2,2002-12-12 12:12:12.123,2001-12-12 12:12:12.123456 -3,2002-12-12 12:12:12.123,2001-12-12 12:12:12.123456 -4,2002-12-12 12:12:12.123,2001-12-12 12:12:12.123456 -5,2003-12-12 12:12:12.123,2001-12-13 12:12:12.123456 -6,2004-12-12 12:12:12.123,2001-12-14 12:12:12.123456 -7,2005-12-12 12:12:12.123,2001-11-12 12:12:12.123456 -8,2006-12-12 12:12:12.123,2001-11-12 12:12:12.123456 -9,2006-12-12 12:12:12.123,2001-11-13 12:12:12.123456 -10,2007-12-12 12:12:12.123,2001-11-14 12:12:12.123456 \ No newline at end of file diff --git a/regression-test/data/partition_p0/auto_partition/auto_partition_stream_load2.csv b/regression-test/data/partition_p0/auto_partition/auto_partition_stream_load2.csv deleted file mode 100644 index 16e3fae4915d650..000000000000000 --- a/regression-test/data/partition_p0/auto_partition/auto_partition_stream_load2.csv +++ /dev/null @@ -1,10 +0,0 @@ -1,Beijing,2001-12-12 12:12:12.123456 -2,BJ,2001-12-12 12:12:12.123456 -3,bj,2001-12-12 12:12:12.123456 -4,bJ,2001-12-12 12:12:12.123456 -5,Chengdu,2001-12-13 12:12:12.123456 -6,XIAN,2001-12-14 12:12:12.123456 -7,Chengdu,2001-11-12 12:12:12.123456 -8,chengDU,2001-11-12 12:12:12.123456 -9,xian,2001-11-13 12:12:12.123456 -10,beiJing,2001-11-14 12:12:12.123456 \ No newline at end of file diff --git a/regression-test/data/partition_p0/auto_partition/test_auto_list_partition.out b/regression-test/data/partition_p0/auto_partition/test_auto_list_partition.out deleted file mode 100644 index 36a6418f3fbc9f5..000000000000000 --- a/regression-test/data/partition_p0/auto_partition/test_auto_list_partition.out +++ /dev/null @@ -1,41 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !sql1 -- -Abc -Beijing -Beijing -XXX -xxx - --- !sql2 -- -Abc -Abc -Beijing -Beijing -Beijing -Beijing -XXX -XXX -new -xxx -xxx - --- !sql3 -- -Abc -Beijing -Beijing -XXX -xxx - --- !sql4 -- -Abc -Abc -Beijing -Beijing -Beijing -Beijing -XXX -XXX -new -xxx -xxx - diff --git a/regression-test/data/partition_p0/auto_partition/test_auto_partition_load.out b/regression-test/data/partition_p0/auto_partition/test_auto_partition_load.out deleted file mode 100644 index 7e1dd673f699277..000000000000000 --- a/regression-test/data/partition_p0/auto_partition/test_auto_partition_load.out +++ /dev/null @@ -1,25 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select1 -- -1 2001-12-12T12:12:12 2001-12-12T12:12:12.123456 -2 2002-12-12T12:12:12 2001-12-12T12:12:12.123456 -3 2002-12-12T12:12:12 2001-12-12T12:12:12.123456 -4 2002-12-12T12:12:12 2001-12-12T12:12:12.123456 -5 2003-12-12T12:12:12 2001-12-13T12:12:12.123456 -6 2004-12-12T12:12:12 2001-12-14T12:12:12.123456 -7 2005-12-12T12:12:12 2001-11-12T12:12:12.123456 -8 2006-12-12T12:12:12 2001-11-12T12:12:12.123456 -9 2006-12-12T12:12:12 2001-11-13T12:12:12.123456 -10 2007-12-12T12:12:12 2001-11-14T12:12:12.123456 - --- !select2 -- -1 Beijing 2001-12-12T12:12:12.123456 -2 BJ 2001-12-12T12:12:12.123456 -3 bj 2001-12-12T12:12:12.123456 -4 bJ 2001-12-12T12:12:12.123456 -5 Chengdu 2001-12-13T12:12:12.123456 -6 XIAN 2001-12-14T12:12:12.123456 -7 Chengdu 2001-11-12T12:12:12.123456 -8 chengDU 2001-11-12T12:12:12.123456 -9 xian 2001-11-13T12:12:12.123456 -10 beiJing 2001-11-14T12:12:12.123456 - diff --git a/regression-test/data/partition_p0/auto_partition/test_auto_range_partition.out b/regression-test/data/partition_p0/auto_partition/test_auto_range_partition.out deleted file mode 100644 index f359c996ecc6820..000000000000000 --- a/regression-test/data/partition_p0/auto_partition/test_auto_range_partition.out +++ /dev/null @@ -1,74 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select00 -- -2022-12-14T00:00 -2022-12-15T00:00 -2022-12-16T00:00 -2022-12-17T00:00 -2022-12-18T00:00 -2022-12-19T00:00 -2022-12-20T00:00 -2122-12-14T00:00 -2122-12-15T00:00 -2122-12-16T00:00 -2122-12-17T00:00 -2122-12-18T00:00 -2122-12-19T00:00 -2122-12-20T00:00 - --- !select01 -- -2022-12-15T00:00 - --- !select02 -- -2022-12-16T00:00 -2022-12-17T00:00 -2022-12-18T00:00 -2022-12-19T00:00 -2022-12-20T00:00 -2122-12-14T00:00 -2122-12-15T00:00 -2122-12-16T00:00 -2122-12-17T00:00 -2122-12-18T00:00 -2122-12-19T00:00 -2122-12-20T00:00 - --- !select10 -- -2022-11-14T22:22:22.222 -2022-11-15T22:22:22.222 -2022-11-16T22:22:22.222 -2022-11-17T22:22:22.222 -2022-11-18T22:22:22.222 -2022-11-19T22:22:22.222 -2022-11-20T22:22:22.222 -2022-12-14T22:22:22.222 -2022-12-15T22:22:22.222 -2022-12-16T22:22:22.222 -2022-12-17T22:22:22.222 -2022-12-18T22:22:22.222 -2022-12-19T22:22:22.222 -2022-12-20T22:22:22.222 -2122-12-14T22:22:22.222 -2122-12-15T22:22:22.222 -2122-12-16T22:22:22.222 -2122-12-17T22:22:22.222 -2122-12-18T22:22:22.222 -2122-12-19T22:22:22.222 -2122-12-20T22:22:22.222 - --- !select11 -- -2022-12-15T22:22:22.222 - --- !select12 -- -2022-12-16T22:22:22.222 -2022-12-17T22:22:22.222 -2022-12-18T22:22:22.222 -2022-12-19T22:22:22.222 -2022-12-20T22:22:22.222 -2122-12-14T22:22:22.222 -2122-12-15T22:22:22.222 -2122-12-16T22:22:22.222 -2122-12-17T22:22:22.222 -2122-12-18T22:22:22.222 -2122-12-19T22:22:22.222 -2122-12-20T22:22:22.222 - diff --git a/regression-test/suites/partition_p0/auto_partition/test_auto_list_partition.groovy b/regression-test/suites/partition_p0/auto_partition/test_auto_list_partition.groovy deleted file mode 100644 index 405b2a1fc5cf77b..000000000000000 --- a/regression-test/suites/partition_p0/auto_partition/test_auto_list_partition.groovy +++ /dev/null @@ -1,91 +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("test_auto_list_partition") { - def tblName1 = "list_table1" - sql "drop table if exists ${tblName1}" - sql """ - CREATE TABLE `${tblName1}` ( - `str` varchar not null - ) ENGINE=OLAP - DUPLICATE KEY(`str`) - COMMENT 'OLAP' - AUTO PARTITION BY LIST (`str`) - ( - ) - DISTRIBUTED BY HASH(`str`) BUCKETS 10 - PROPERTIES ( - "replication_allocation" = "tag.location.default: 1" - ); - """ - sql """ insert into ${tblName1} values ("Beijing"), ("XXX"), ("xxx"), ("Beijing"), ("Abc") """ - qt_sql1 """ select * from ${tblName1} order by `str` """ - result11 = sql "show partitions from ${tblName1}" - assertEquals(result11.size(), 4) - sql """ insert into ${tblName1} values ("Beijing"), ("XXX"), ("xxx"), ("Beijing"), ("Abc"), ("new") """ - qt_sql2 """ select * from ${tblName1} order by `str` """ - result12 = sql "show partitions from ${tblName1}" - assertEquals(result12.size(), 5) - - def tblName2 = "list_table2" - sql "drop table if exists ${tblName2}" - sql """ - CREATE TABLE `${tblName2}` ( - `str` varchar not null - ) ENGINE=OLAP - DUPLICATE KEY(`str`) - COMMENT 'OLAP' - AUTO PARTITION BY LIST (`str`) - ( - ) - DISTRIBUTED BY HASH(`str`) BUCKETS 10 - PROPERTIES ( - "replication_allocation" = "tag.location.default: 1" - ); - """ - sql """ insert into ${tblName2} values ("Beijing"), ("XXX"), ("xxx"), ("Beijing"), ("Abc") """ - qt_sql3 """ select * from ${tblName2} order by `str` """ - result21 = sql "show partitions from ${tblName2}" - assertEquals(result21.size(), 4) - sql """ insert into ${tblName2} values ("Beijing"), ("XXX"), ("xxx"), ("Beijing"), ("Abc"), ("new") """ - qt_sql4 """ select * from ${tblName2} order by `str` """ - result22 = sql "show partitions from ${tblName2}" - assertEquals(result22.size(), 5) - - def tblName3 = "list_table3" - sql "drop table if exists ${tblName3}" - sql """ - CREATE TABLE `${tblName3}` ( - `k1` INT, - `k2` VARCHAR(50) not null, - `k3` DATETIMEV2(6) - ) ENGINE=OLAP - DUPLICATE KEY(`k1`) - COMMENT 'OLAP' - AUTO PARTITION BY LIST (`k2`) - ( - ) - DISTRIBUTED BY HASH(`k1`) BUCKETS 16 - PROPERTIES ( - "replication_allocation" = "tag.location.default: 1" - ); - """ - sql """ insert into ${tblName3} values (1, 'ABC', '2000-01-01 12:12:12.123456'), (2, 'AAA', '2000-01-01'), (3, 'aaa', '2000-01-01'), (3, 'AaA', '2000-01-01') """ - result3 = sql "show partitions from ${tblName3}" - logger.info("${result3}") - assertEquals(result3.size(), 4) -} diff --git a/regression-test/suites/partition_p0/auto_partition/test_auto_partition_load.groovy b/regression-test/suites/partition_p0/auto_partition/test_auto_partition_load.groovy deleted file mode 100644 index 0cf2eaf9c12dc27..000000000000000 --- a/regression-test/suites/partition_p0/auto_partition/test_auto_partition_load.groovy +++ /dev/null @@ -1,79 +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("test_auto_partition_load") { - def tblName1 = "load_table1" - sql "drop table if exists ${tblName1}" - sql """ - CREATE TABLE `${tblName1}` ( - `k1` INT, - `k2` DATETIME, - `k3` DATETIMEV2(6) - ) ENGINE=OLAP - DUPLICATE KEY(`k1`) - COMMENT 'OLAP' - AUTO PARTITION BY RANGE date_trunc(`k2`, 'year') - ( - ) - DISTRIBUTED BY HASH(`k1`) BUCKETS 16 - PROPERTIES ( - "replication_allocation" = "tag.location.default: 1" - ); - """ - streamLoad { - table "${tblName1}" - set 'column_separator', ',' - file "auto_partition_stream_load1.csv" - time 20000 - } - - qt_select1 "select * from ${tblName1} order by k1" - result1 = sql "show partitions from ${tblName1}" - logger.info("${result1}") - assertEquals(result1.size(), 7) - - - def tblName2 = "load_table2" - sql "drop table if exists ${tblName2}" - sql """ - CREATE TABLE `${tblName2}` ( - `k1` INT, - `k2` VARCHAR(50) not null, - `k3` DATETIMEV2(6) - ) ENGINE=OLAP - DUPLICATE KEY(`k1`) - COMMENT 'OLAP' - AUTO PARTITION BY LIST (`k2`) - ( - ) - DISTRIBUTED BY HASH(`k1`) BUCKETS 16 - PROPERTIES ( - "replication_allocation" = "tag.location.default: 1" - ); - """ - streamLoad { - table "${tblName2}" - set 'column_separator', ',' - file "auto_partition_stream_load2.csv" - time 20000 - } - - qt_select2 "select * from ${tblName2} order by k1" - result2 = sql "show partitions from ${tblName2}" - logger.info("${result2}") - assertEquals(result2.size(), 9) -} diff --git a/regression-test/suites/partition_p0/auto_partition/test_auto_range_partition.groovy b/regression-test/suites/partition_p0/auto_partition/test_auto_range_partition.groovy deleted file mode 100644 index 874704ea8f5e214..000000000000000 --- a/regression-test/suites/partition_p0/auto_partition/test_auto_range_partition.groovy +++ /dev/null @@ -1,90 +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("test_auto_range_partition") { - def tblName1 = "range_table1" - sql "drop table if exists ${tblName1}" - // not support datev2 now. need impl date_trunc(datev2) - sql """ - CREATE TABLE `${tblName1}` ( - `TIME_STAMP` datetimev2 NOT NULL COMMENT '采集日期' - ) ENGINE=OLAP - DUPLICATE KEY(`TIME_STAMP`) - COMMENT 'OLAP' - AUTO PARTITION BY RANGE date_trunc(`TIME_STAMP`, 'day') - ( - ) - DISTRIBUTED BY HASH(`TIME_STAMP`) BUCKETS 10 - PROPERTIES ( - "replication_allocation" = "tag.location.default: 1" - ); - """ - sql """ insert into ${tblName1} values ('2022-12-14'), ('2022-12-15'), ('2022-12-16'), ('2022-12-17'), ('2022-12-18'), ('2022-12-19'), ('2022-12-20') """ - sql """ insert into ${tblName1} values ('2122-12-14'), ('2122-12-15'), ('2122-12-16'), ('2122-12-17'), ('2122-12-18'), ('2122-12-19'), ('2122-12-20') """ - - qt_select00 """ select * from ${tblName1} order by TIME_STAMP """ - qt_select01 """ select * from ${tblName1} WHERE TIME_STAMP = '2022-12-15' order by TIME_STAMP """ - qt_select02 """ select * from ${tblName1} WHERE TIME_STAMP > '2022-12-15' order by TIME_STAMP """ - - def tblName2 = "range_table2" - sql "drop table if exists ${tblName2}" - sql """ - CREATE TABLE `${tblName2}` ( - `TIME_STAMP` datetimev2(3) NOT NULL COMMENT '采集日期' - ) ENGINE=OLAP - DUPLICATE KEY(`TIME_STAMP`) - COMMENT 'OLAP' - AUTO PARTITION BY RANGE date_trunc(`TIME_STAMP`, 'day') - ( - ) - DISTRIBUTED BY HASH(`TIME_STAMP`) BUCKETS 10 - PROPERTIES ( - "replication_allocation" = "tag.location.default: 1" - ); - """ - sql """ insert into ${tblName2} values ('2022-12-14 22:22:22.222'), ('2022-12-15 22:22:22.222'), ('2022-12-16 22:22:22.222'), ('2022-12-17 22:22:22.222'), ('2022-12-18 22:22:22.222'), ('2022-12-19 22:22:22.222'), ('2022-12-20 22:22:22.222') """ - sql """ insert into ${tblName2} values ('2122-12-14 22:22:22.222'), ('2122-12-15 22:22:22.222'), ('2122-12-16 22:22:22.222'), ('2122-12-17 22:22:22.222'), ('2122-12-18 22:22:22.222'), ('2122-12-19 22:22:22.222'), ('2122-12-20 22:22:22.222') """ - sql """ insert into ${tblName2} values ('2022-11-14 22:22:22.222'), ('2022-11-15 22:22:22.222'), ('2022-11-16 22:22:22.222'), ('2022-11-17 22:22:22.222'), ('2022-11-18 22:22:22.222'), ('2022-11-19 22:22:22.222'), ('2022-11-20 22:22:22.222') """ - - - qt_select10 """ select * from ${tblName2} order by TIME_STAMP """ - qt_select11 """ select * from ${tblName2} WHERE TIME_STAMP = '2022-12-15 22:22:22.222' order by TIME_STAMP """ - qt_select12 """ select * from ${tblName2} WHERE TIME_STAMP > '2022-12-15 22:22:22.222' order by TIME_STAMP """ - - def tblName3 = "range_table3" - sql "drop table if exists ${tblName3}" - sql """ - CREATE TABLE `${tblName3}` ( - `k1` INT, - `k2` DATETIMEV2(3), - `k3` DATETIMEV2(6) - ) ENGINE=OLAP - DUPLICATE KEY(`k1`) - COMMENT 'OLAP' - AUTO PARTITION BY RANGE date_trunc(`k2`, 'day') - ( - ) - DISTRIBUTED BY HASH(`k1`) BUCKETS 16 - PROPERTIES ( - "replication_allocation" = "tag.location.default: 1" - ); - """ - sql """ insert into ${tblName3} values (1, '1990-01-01', '2000-01-01 12:12:12.123456'), (2, '1991-02-01', '2000-01-01'), (3, '1991-01-01', '2000-01-01'), (3, '1991-01-01', '2000-01-01') """ - result1 = sql "show partitions from ${tblName3}" - logger.info("${result1}") - assertEquals(result1.size(), 3) -} diff --git a/regression-test/suites/partition_p0/test_datev2_partition.groovy b/regression-test/suites/partition_p0/test_datev2_partition.groovy index 63852bb4e2e6726..600b820684c2402 100644 --- a/regression-test/suites/partition_p0/test_datev2_partition.groovy +++ b/regression-test/suites/partition_p0/test_datev2_partition.groovy @@ -43,6 +43,7 @@ suite("test_datev2_partition") { qt_select """ select * from ${tblName1} order by TIME_STAMP """ qt_select """ select * from ${tblName1} WHERE TIME_STAMP = '2022-12-15' order by TIME_STAMP """ qt_select """ select * from ${tblName1} WHERE TIME_STAMP > '2022-12-15' order by TIME_STAMP """ + sql "drop table ${tblName1}" def tblName2 = "test_datev2_partition2" sql "drop table if exists ${tblName2}" @@ -71,4 +72,5 @@ suite("test_datev2_partition") { qt_select """ select * from ${tblName2} order by TIME_STAMP """ qt_select """ select * from ${tblName2} WHERE TIME_STAMP = '2022-12-15 22:22:22.222' order by TIME_STAMP """ qt_select """ select * from ${tblName2} WHERE TIME_STAMP > '2022-12-15 22:22:22.222' order by TIME_STAMP """ + sql "drop table ${tblName2}" }