diff --git a/be/src/exec/tablet_info.cpp b/be/src/exec/tablet_info.cpp index 2bec0113ace22e..c594b6ad7c6424 100644 --- a/be/src/exec/tablet_info.cpp +++ b/be/src/exec/tablet_info.cpp @@ -23,14 +23,16 @@ #include #include #include -#include #include +#include +#include #include #include #include #include "common/exception.h" +#include "common/logging.h" #include "common/status.h" #include "olap/tablet_schema.h" #include "runtime/define_primitive_type.h" @@ -40,13 +42,11 @@ #include "runtime/primitive_type.h" #include "runtime/raw_value.h" #include "runtime/types.h" -#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" +// NOLINTNEXTLINE(unused-includes) +#include "vec/exprs/vexpr_context.h" // IWYU pragma: keep #include "vec/exprs/vliteral.h" #include "vec/runtime/vdatetime_value.h" @@ -55,19 +55,19 @@ namespace doris { void OlapTableIndexSchema::to_protobuf(POlapTableIndexSchema* pindex) const { pindex->set_id(index_id); pindex->set_schema_hash(schema_hash); - for (auto slot : slots) { + for (auto* slot : slots) { pindex->add_columns(slot->col_name()); } - for (auto column : columns) { + for (auto* column : columns) { column->to_schema_pb(pindex->add_columns_desc()); } - for (auto index : indexes) { + for (auto* index : indexes) { index->to_schema_pb(pindex->add_indexes_desc()); } } -bool VOlapTablePartKeyComparator::operator()(const BlockRowWithIndicator lhs, - const BlockRowWithIndicator rhs) 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); @@ -101,15 +101,6 @@ bool VOlapTablePartKeyComparator::operator()(const BlockRowWithIndicator lhs, 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) { @@ -132,14 +123,14 @@ Status OlapTableSchemaParam::init(const POlapTableSchemaParam& pschema) { _auto_increment_column = pschema.auto_increment_column(); } - for (auto& col : pschema.partial_update_input_columns()) { + for (const auto& col : pschema.partial_update_input_columns()) { _partial_update_input_columns.insert(col); } std::unordered_map, SlotDescriptor*> slots_map; _tuple_desc = _obj_pool.add(new TupleDescriptor(pschema.tuple_desc())); - for (auto& p_slot_desc : pschema.slot_descs()) { - auto slot_desc = _obj_pool.add(new SlotDescriptor(p_slot_desc)); + for (const auto& p_slot_desc : pschema.slot_descs()) { + auto* slot_desc = _obj_pool.add(new SlotDescriptor(p_slot_desc)); _tuple_desc->add_slot(slot_desc); string data_type; EnumToString(TPrimitiveType, to_thrift(slot_desc->col_type()), data_type); @@ -148,13 +139,13 @@ Status OlapTableSchemaParam::init(const POlapTableSchemaParam& pschema) { slot_desc); } - for (auto& p_index : pschema.indexes()) { - auto index = _obj_pool.add(new OlapTableIndexSchema()); + for (const auto& p_index : pschema.indexes()) { + auto* index = _obj_pool.add(new OlapTableIndexSchema()); index->index_id = p_index.id(); index->schema_hash = p_index.schema_hash(); - for (auto& pcolumn_desc : p_index.columns_desc()) { + for (const auto& pcolumn_desc : p_index.columns_desc()) { if (!_is_partial_update || - _partial_update_input_columns.count(pcolumn_desc.name()) > 0) { + _partial_update_input_columns.contains(pcolumn_desc.name())) { auto it = slots_map.find(std::make_pair( to_lower(pcolumn_desc.name()), TabletColumn::get_field_type_by_string(pcolumn_desc.type()))); @@ -168,7 +159,7 @@ Status OlapTableSchemaParam::init(const POlapTableSchemaParam& pschema) { tc->init_from_pb(pcolumn_desc); index->columns.emplace_back(tc); } - for (auto& pindex_desc : p_index.indexes_desc()) { + for (const auto& pindex_desc : p_index.indexes_desc()) { TabletIndex* ti = _obj_pool.add(new TabletIndex()); ti->init_from_pb(pindex_desc); index->indexes.emplace_back(ti); @@ -195,26 +186,26 @@ Status OlapTableSchemaParam::init(const TOlapTableSchemaParam& tschema) { _auto_increment_column = tschema.auto_increment_column; } - for (auto& tcolumn : tschema.partial_update_input_columns) { + for (const auto& tcolumn : tschema.partial_update_input_columns) { _partial_update_input_columns.insert(tcolumn); } std::unordered_map, SlotDescriptor*> slots_map; _tuple_desc = _obj_pool.add(new TupleDescriptor(tschema.tuple_desc)); - for (auto& t_slot_desc : tschema.slot_descs) { - auto slot_desc = _obj_pool.add(new SlotDescriptor(t_slot_desc)); + for (const auto& t_slot_desc : tschema.slot_descs) { + auto* slot_desc = _obj_pool.add(new SlotDescriptor(t_slot_desc)); _tuple_desc->add_slot(slot_desc); slots_map.emplace(std::make_pair(to_lower(slot_desc->col_name()), slot_desc->col_type()), slot_desc); } - for (auto& t_index : tschema.indexes) { + for (const auto& t_index : tschema.indexes) { std::unordered_map index_slots_map; - auto index = _obj_pool.add(new OlapTableIndexSchema()); + auto* index = _obj_pool.add(new OlapTableIndexSchema()); index->index_id = t_index.id; index->schema_hash = t_index.schema_hash; - for (auto& tcolumn_desc : t_index.columns_desc) { + for (const auto& tcolumn_desc : t_index.columns_desc) { if (!_is_partial_update || - _partial_update_input_columns.count(tcolumn_desc.column_name) > 0) { + _partial_update_input_columns.contains(tcolumn_desc.column_name)) { auto it = slots_map.find( std::make_pair(to_lower(tcolumn_desc.column_name), thrift_to_type(tcolumn_desc.column_type.type))); @@ -231,7 +222,7 @@ Status OlapTableSchemaParam::init(const TOlapTableSchemaParam& tschema) { index->columns.emplace_back(tc); } if (t_index.__isset.indexes_desc) { - for (auto& tindex_desc : t_index.indexes_desc) { + for (const auto& tindex_desc : t_index.indexes_desc) { std::vector column_unique_ids(tindex_desc.columns.size()); for (size_t i = 0; i < tindex_desc.columns.size(); i++) { auto it = index_slots_map.find(to_lower(tindex_desc.columns[i])); @@ -269,10 +260,10 @@ void OlapTableSchemaParam::to_protobuf(POlapTableSchemaParam* pschema) const { *pschema->add_partial_update_input_columns() = col; } _tuple_desc->to_protobuf(pschema->mutable_tuple_desc()); - for (auto slot : _tuple_desc->slots()) { + for (auto* slot : _tuple_desc->slots()) { slot->to_protobuf(pschema->add_slot_descs()); } - for (auto index : _indexes) { + for (auto* index : _indexes) { index->to_protobuf(pschema->add_indexes()); } } @@ -290,11 +281,6 @@ VOlapTablePartitionParam::VOlapTablePartitionParam(std::shared_ptrtuple_desc()->slots()), _mem_tracker(std::make_unique("OlapTablePartitionParam")), _part_type(t_param.partition_type) { - 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_partition = true; auto size = t_param.partition_function_exprs.size(); @@ -314,6 +300,45 @@ VOlapTablePartitionParam::VOlapTablePartitionParam(std::shared_ptrroot(); } } + + if (t_param.__isset.enable_auto_detect_overwrite && t_param.enable_auto_detect_overwrite) { + _is_auto_detect_overwrite = true; + DCHECK(t_param.__isset.overwrite_group_id); + _overwrite_group_id = t_param.overwrite_group_id; + } + + if (_is_auto_partition) { + // the nullable mode depends on partition_exprs. not column slots. so use them. + DCHECK(_partition_function.size() <= _slots.size()) + << _partition_function.size() << ", " << _slots.size(); + + // suppose (k0, [k1], [k2]), so get [k1, 0], [k2, 1] + std::map partition_slots_map; // name to idx in part_exprs + for (size_t i = 0; i < t_param.partition_columns.size(); i++) { + partition_slots_map.emplace(t_param.partition_columns[i], i); + } + + // here we rely on the same order and number of the _part_funcs and _slots in the prefix + // _part_block contains all slots of table. + for (auto* slot : _slots) { + // try to replace with partition expr. + if (auto it = partition_slots_map.find(slot->col_name()); + it != partition_slots_map.end()) { // it's a partition column slot + auto& expr_type = _partition_function[it->second]->data_type(); + _partition_block.insert({expr_type->create_column(), expr_type, slot->col_name()}); + } else { + _partition_block.insert({slot->get_empty_mutable_column(), + slot->get_data_type_ptr(), slot->col_name()}); + } + } + VLOG_TRACE << _partition_block.dump_structure(); + } else { + // we insert all. but not all will be used. it will controlled by _partition_slot_locs + for (auto* slot : _slots) { + _partition_block.insert({slot->get_empty_mutable_column(), slot->get_data_type_ptr(), + slot->col_name()}); + } + } } VOlapTablePartitionParam::~VOlapTablePartitionParam() { @@ -322,7 +347,7 @@ VOlapTablePartitionParam::~VOlapTablePartitionParam() { Status VOlapTablePartitionParam::init() { std::vector slot_column_names; - for (auto slot_desc : _schema->tuple_desc()->slots()) { + for (auto* slot_desc : _schema->tuple_desc()->slots()) { slot_column_names.emplace_back(slot_desc->col_name()); } @@ -337,6 +362,7 @@ Status VOlapTablePartitionParam::init() { return Status::OK(); }; + // here we find the partition columns. others maybe non-partition columns/special columns. if (_t_param.__isset.partition_columns) { for (auto& part_col : _t_param.partition_columns) { RETURN_IF_ERROR(find_slot_locs(part_col, _partition_slot_locs, "partition")); @@ -377,82 +403,18 @@ Status VOlapTablePartitionParam::init() { 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}); -} - -Status VOlapTablePartitionParam::_create_partition_keys(const std::vector& t_exprs, - BlockRow* part_key) { - for (int i = 0; i < t_exprs.size(); i++) { - RETURN_IF_ERROR(_create_partition_key(t_exprs[i], part_key, _partition_slot_locs[i])); - } - return Status::OK(); + // we have used upper_bound to find to ensure key < part.right and this part is closest(right - key is min) + // now we only have to check (key >= part.left). the comparator(a,b) means a < b, so we use anti + return part->start_key.second == -1 /* spj: start_key.second == -1 means only single partition*/ + || !comparator(key, std::tuple {part->start_key.first, part->start_key.second, false}); } -Status VOlapTablePartitionParam::generate_partition_from(const TOlapTablePartition& t_part, - VOlapTablePartition*& part_result) { - DCHECK(part_result == nullptr); - part_result = _obj_pool.add(new VOlapTablePartition(&_partition_block)); - part_result->id = t_part.id; - part_result->is_mutable = t_part.is_mutable; - // only load_to_single_tablet = true will set load_tablet_idx - if (t_part.__isset.load_tablet_idx) { - part_result->load_tablet_idx = t_part.load_tablet_idx; - } - - 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) { +// insert value into _partition_block's column +// NOLINTBEGIN(readability-function-size) +static Status _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(); + //TODO: use assert_cast before insert_data switch (t_expr.node_type) { case TExprNodeType::DATE_LITERAL: { if (TypeDescriptor::from_thrift(t_expr.type).is_date_v2_type()) { @@ -512,9 +474,9 @@ Status VOlapTablePartitionParam::_create_partition_key(const TExprNode& t_expr, } case TExprNodeType::LARGE_INT_LITERAL: { StringParser::ParseResult parse_result = StringParser::PARSE_SUCCESS; - __int128 value = StringParser::string_to_int<__int128>( - t_expr.large_int_literal.value.c_str(), t_expr.large_int_literal.value.size(), - &parse_result); + auto value = StringParser::string_to_int<__int128>(t_expr.large_int_literal.value.c_str(), + t_expr.large_int_literal.value.size(), + &parse_result); if (parse_result != StringParser::PARSE_SUCCESS) { value = MAX_INT128; } @@ -544,16 +506,80 @@ Status VOlapTablePartitionParam::_create_partition_key(const TExprNode& t_expr, part_key->second = column->size() - 1; return Status::OK(); } +// NOLINTEND(readability-function-size) + +Status VOlapTablePartitionParam::_create_partition_keys(const std::vector& t_exprs, + BlockRow* part_key) { + for (int i = 0; i < t_exprs.size(); i++) { + RETURN_IF_ERROR(_create_partition_key(t_exprs[i], part_key, _partition_slot_locs[i])); + } + return Status::OK(); +} + +Status VOlapTablePartitionParam::generate_partition_from(const TOlapTablePartition& t_part, + VOlapTablePartition*& part_result) { + DCHECK(part_result == nullptr); + // here we set the default value of partition bounds first! if it doesn't have some key, it will be -1. + part_result = _obj_pool.add(new VOlapTablePartition(&_partition_block)); + part_result->id = t_part.id; + part_result->is_mutable = t_part.is_mutable; + // only load_to_single_tablet = true will set load_tablet_idx + if (t_part.__isset.load_tablet_idx) { + part_result->load_tablet_idx = t_part.load_tablet_idx; + } + + if (_is_in_partition) { + 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; + } + } else { // range + if (t_part.__isset.start_keys) { + RETURN_IF_ERROR(_create_partition_keys(t_part.start_keys, &part_result->start_key)); + } + // we generate the right bound but not insert into partition map + if (t_part.__isset.end_keys) { + RETURN_IF_ERROR(_create_partition_keys(t_part.end_keys, &part_result->end_key)); + } + } + + 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) { + 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::add_partitions( const std::vector& partitions) { for (const auto& t_part : partitions) { - auto part = _obj_pool.add(new VOlapTablePartition(&_partition_block)); + 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); + // we dont pass right keys when it's MAX_VALUE. so there's possibility we only have start_key but not end_key // range partition if (t_part.__isset.start_keys) { RETURN_IF_ERROR(_create_partition_keys(t_part.start_keys, &part->start_key)); @@ -588,10 +614,6 @@ Status VOlapTablePartitionParam::add_partitions( // 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={}", @@ -613,4 +635,88 @@ Status VOlapTablePartitionParam::add_partitions( return Status::OK(); } +Status VOlapTablePartitionParam::replace_partitions( + std::vector& old_partition_ids, + const std::vector& new_partitions) { + // remove old replaced partitions + DCHECK(old_partition_ids.size() == new_partitions.size()); + + // init and add new partitions. insert into _partitions + for (int i = 0; i < new_partitions.size(); i++) { + const auto& t_part = new_partitions[i]; + // pair old_partition_ids and new_partitions one by one. TODO: sort to opt performance + VOlapTablePartition* old_part = nullptr; + auto old_part_id = old_partition_ids[i]; + if (auto it = std::find_if( + _partitions.begin(), _partitions.end(), + [=](const VOlapTablePartition* lhs) { return lhs->id == old_part_id; }); + it != _partitions.end()) { + old_part = *it; + } else { + return Status::InternalError("Cannot find old tablet {} in replacing", old_part_id); + } + + auto* part = _obj_pool.add(new VOlapTablePartition(&_partition_block)); + part->id = t_part.id; + part->is_mutable = t_part.is_mutable; + + /// just substitute directly. no need to remove and reinsert keys. + // range partition + part->start_key = std::move(old_part->start_key); + part->end_key = std::move(old_part->end_key); + // list partition + part->in_keys = std::move(old_part->in_keys); + 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) { + 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); + } + } + + // add new partitions with new id. + _partitions.emplace_back(part); + + // replace items in _partition_maps + if (_is_in_partition) { + for (auto& in_key : part->in_keys) { + (*_partitions_map)[std::tuple {in_key.first, in_key.second, false}] = part; + } + } else { + (*_partitions_map)[std::tuple {part->end_key.first, part->end_key.second, false}] = + part; + } + } + // remove old partitions by id + std::ranges::sort(old_partition_ids); + for (auto it = _partitions.begin(); it != _partitions.end();) { + if (std::ranges::binary_search(old_partition_ids, (*it)->id)) { + it = _partitions.erase(it); + } else { + it++; + } + } + + return Status::OK(); +} + } // namespace doris diff --git a/be/src/exec/tablet_info.h b/be/src/exec/tablet_info.h index a157b57948721f..092d9a18c7b2a6 100644 --- a/be/src/exec/tablet_info.h +++ b/be/src/exec/tablet_info.h @@ -32,6 +32,7 @@ #include #include +#include "common/logging.h" #include "common/object_pool.h" #include "common/status.h" #include "runtime/descriptors.h" @@ -40,7 +41,6 @@ #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 { @@ -119,7 +119,7 @@ using OlapTableIndexTablets = TOlapTableIndexTablets; using BlockRow = std::pair; using BlockRowWithIndicator = - std::tuple; // [block, column, is_transformed] + std::tuple; // [block, row, is_transformed] struct VOlapTablePartition { int64_t id = 0; @@ -133,6 +133,7 @@ struct VOlapTablePartition { int64_t load_tablet_idx = -1; VOlapTablePartition(vectorized::Block* partition_block) + // the default value of partition bound is -1. : start_key {partition_block, -1}, end_key {partition_block, -1} {} }; @@ -145,7 +146,7 @@ class VOlapTablePartKeyComparator { // return true if lhs < rhs // 'row' is -1 mean maximal boundary - bool operator()(const BlockRowWithIndicator lhs, const BlockRowWithIndicator rhs) const; + bool operator()(const BlockRowWithIndicator& lhs, const BlockRowWithIndicator& rhs) const; private: const std::vector& _slot_locs; @@ -167,11 +168,14 @@ class VOlapTablePartitionParam { int64_t version() const { return _t_param.version; } // return true if we found this block_row in partition - //TODO: use virtual function to refactor it ALWAYS_INLINE bool find_partition(vectorized::Block* block, int row, VOlapTablePartition*& partition) const { auto it = _is_in_partition ? _partitions_map->find(std::tuple {block, row, true}) : _partitions_map->upper_bound(std::tuple {block, row, true}); + VLOG_TRACE << "find row " << row << " of\n" + << block->dump_data() << "in:\n" + << _partition_block.dump_data() << "result line row: " << std::get<1>(it->first); + // for list partition it might result in default partition if (_is_in_partition) { partition = (it != _partitions_map->end()) ? it->second : _default_partition; @@ -246,9 +250,15 @@ class VOlapTablePartitionParam { bool is_projection_partition() const { return _is_auto_partition; } bool is_auto_partition() const { return _is_auto_partition; } + bool is_auto_detect_overwrite() const { return _is_auto_detect_overwrite; } + int64_t get_overwrite_group_id() const { return _overwrite_group_id; } + std::vector get_partition_keys() const { return _partition_slot_locs; } Status add_partitions(const std::vector& partitions); + // no need to del/reinsert partition keys, but change the link. reset the _partitions items + Status replace_partitions(std::vector& old_partition_ids, + const std::vector& new_partitions); vectorized::VExprContextSPtrs get_part_func_ctx() { return _part_func_ctx; } vectorized::VExprSPtrs get_partition_function() { return _partition_function; } @@ -264,8 +274,6 @@ class VOlapTablePartitionParam { private: Status _create_partition_keys(const std::vector& t_exprs, BlockRow* part_key); - Status _create_partition_key(const TExprNode& t_expr, BlockRow* part_key, uint16_t pos); - // check if this partition contain this key bool _part_contains(VOlapTablePartition* part, BlockRowWithIndicator key) const; @@ -284,6 +292,7 @@ class VOlapTablePartitionParam { 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). + // For range partition we ONLY SAVE RIGHT ENDS. when we find a part's RIGHT by a value, check if part's left cover it then. std::unique_ptr< std::map> _partitions_map; @@ -293,11 +302,13 @@ class VOlapTablePartitionParam { // 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_partition = false; vectorized::VExprContextSPtrs _part_func_ctx = {nullptr}; vectorized::VExprSPtrs _partition_function = {nullptr}; TPartitionType::type _part_type; // support list or range + // "insert overwrite partition(*)", detect which partitions by BE + bool _is_auto_detect_overwrite = false; + int64_t _overwrite_group_id = 0; }; // indicate where's the tablet and all its replications (node-wise) @@ -360,13 +371,13 @@ class DorisNodesInfo { public: DorisNodesInfo() = default; DorisNodesInfo(const TPaloNodesInfo& t_nodes) { - for (auto& node : t_nodes.nodes) { + for (const auto& node : t_nodes.nodes) { _nodes.emplace(node.id, node); } } void setNodes(const TPaloNodesInfo& t_nodes) { _nodes.clear(); - for (auto& node : t_nodes.nodes) { + for (const auto& node : t_nodes.nodes) { _nodes.emplace(node.id, node); } } @@ -380,7 +391,7 @@ class DorisNodesInfo { void add_nodes(const std::vector& t_nodes) { for (const auto& node : t_nodes) { - auto node_info = find_node(node.id); + const auto* node_info = find_node(node.id); if (node_info == nullptr) { _nodes.emplace(node.id, node); } diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp index f8df3fd0a665d4..5b8d4945ad10ff 100644 --- a/be/src/olap/storage_engine.cpp +++ b/be/src/olap/storage_engine.cpp @@ -662,6 +662,10 @@ void StorageEngine::stop() { _cold_data_compaction_thread_pool->shutdown(); } + if (_cooldown_thread_pool) { + _cooldown_thread_pool->shutdown(); + } + _memtable_flush_executor.reset(nullptr); _calc_delete_bitmap_executor.reset(nullptr); diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index f96a8f2dc215df..688e12d97221aa 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -177,30 +177,6 @@ void set_last_failure_time(Tablet* tablet, const Compaction& compaction, int64_t } // namespace -struct WriteCooldownMetaExecutors { - WriteCooldownMetaExecutors(size_t executor_nums = 5); - - static WriteCooldownMetaExecutors* get_instance() { - static WriteCooldownMetaExecutors instance; - return &instance; - } - - void submit(TabletSharedPtr tablet); - size_t _get_executor_pos(int64_t tablet_id) const { - return std::hash()(tablet_id) % _executor_nums; - }; - // Each executor is a mpsc to ensure uploads of the same tablet meta are not concurrent - // FIXME(AlexYue): Use mpsc instead of `ThreadPool` with 1 thread - // We use PriorityThreadPool since it would call status inside it's `shutdown` function. - // Consider one situation where the StackTraceCache's singleton is detructed before - // this WriteCooldownMetaExecutors's singleton, then invoking the status would also call - // StackTraceCache which would then result in heap use after free like #23834 - std::vector> _executors; - std::unordered_set _pending_tablets; - std::mutex _latch; - size_t _executor_nums; -}; - WriteCooldownMetaExecutors::WriteCooldownMetaExecutors(size_t executor_nums) : _executor_nums(executor_nums) { for (size_t i = 0; i < _executor_nums; i++) { @@ -214,6 +190,14 @@ WriteCooldownMetaExecutors::WriteCooldownMetaExecutors(size_t executor_nums) } } +void WriteCooldownMetaExecutors::stop() { + for (auto& pool_ptr : _executors) { + if (pool_ptr) { + pool_ptr->shutdown(); + } + } +} + void WriteCooldownMetaExecutors::WriteCooldownMetaExecutors::submit(TabletSharedPtr tablet) { auto tablet_id = tablet->tablet_id(); @@ -2217,7 +2201,7 @@ Status check_version_continuity(const std::vector& rs_metas // It's guaranteed the write cooldown meta task would be invoked at the end unless BE crashes // one tablet would at most have one async task to be done void Tablet::async_write_cooldown_meta(TabletSharedPtr tablet) { - WriteCooldownMetaExecutors::get_instance()->submit(std::move(tablet)); + ExecEnv::GetInstance()->write_cooldown_meta_executors()->submit(std::move(tablet)); } bool Tablet::update_cooldown_conf(int64_t cooldown_term, int64_t cooldown_replica_id) { diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index bf6730fe78e1c9..8b28b57897d3e4 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -24,9 +24,6 @@ #include #include #include -#include -#include -#include #include #include #include @@ -34,7 +31,6 @@ #include #include #include -#include #include #include @@ -86,6 +82,27 @@ enum TabletStorageType { STORAGE_TYPE_LOCAL, STORAGE_TYPE_REMOTE, STORAGE_TYPE_R static inline constexpr auto TRACE_TABLET_LOCK_THRESHOLD = std::chrono::seconds(1); +struct WriteCooldownMetaExecutors { + WriteCooldownMetaExecutors(size_t executor_nums = 5); + + void stop(); + + void submit(TabletSharedPtr tablet); + size_t _get_executor_pos(int64_t tablet_id) const { + return std::hash()(tablet_id) % _executor_nums; + }; + // Each executor is a mpsc to ensure uploads of the same tablet meta are not concurrent + // FIXME(AlexYue): Use mpsc instead of `ThreadPool` with 1 thread + // We use PriorityThreadPool since it would call status inside it's `shutdown` function. + // Consider one situation where the StackTraceCache's singleton is detructed before + // this WriteCooldownMetaExecutors's singleton, then invoking the status would also call + // StackTraceCache which would then result in heap use after free like #23834 + std::vector> _executors; + std::unordered_set _pending_tablets; + std::mutex _latch; + size_t _executor_nums; +}; + class Tablet final : public BaseTablet { public: Tablet(StorageEngine& engine, TabletMetaSharedPtr tablet_meta, DataDir* data_dir, diff --git a/be/src/olap/utils.h b/be/src/olap/utils.h index d078d260f09f0a..4539aef288f005 100644 --- a/be/src/olap/utils.h +++ b/be/src/olap/utils.h @@ -261,15 +261,14 @@ constexpr bool is_numeric_type(const FieldType& field_type) { } // Util used to get string name of thrift enum item -#define EnumToString(enum_type, index, out) \ - do { \ - std::map::const_iterator it = \ - _##enum_type##_VALUES_TO_NAMES.find(index); \ - if (it == _##enum_type##_VALUES_TO_NAMES.end()) { \ - out = "NULL"; \ - } else { \ - out = it->second; \ - } \ +#define EnumToString(enum_type, index, out) \ + do { \ + auto it = _##enum_type##_VALUES_TO_NAMES.find(index); \ + if (it == _##enum_type##_VALUES_TO_NAMES.end()) { \ + out = "NULL"; \ + } else { \ + out = it->second; \ + } \ } while (0) struct RowLocation { diff --git a/be/src/runtime/exec_env.cpp b/be/src/runtime/exec_env.cpp index 6ecd359c108e6c..a2a031c7226df5 100644 --- a/be/src/runtime/exec_env.cpp +++ b/be/src/runtime/exec_env.cpp @@ -30,7 +30,6 @@ #include "olap/tablet_manager.h" #include "runtime/fragment_mgr.h" #include "runtime/frontend_info.h" -#include "time.h" #include "util/debug_util.h" #include "util/time.h" #include "vec/sink/delta_writer_v2_pool.h" @@ -45,6 +44,15 @@ ExecEnv::~ExecEnv() { } // TODO(plat1ko): template +#ifdef BE_TEST +void ExecEnv::set_storage_engine(std::unique_ptr&& engine) { + _storage_engine = std::move(engine); +} +void ExecEnv::set_write_cooldown_meta_executors() { + _write_cooldown_meta_executors = std::make_unique(); +} +#endif // BE_TEST + Result ExecEnv::get_tablet(int64_t tablet_id) { BaseTabletSPtr tablet; std::string err; diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 9785613a48d557..4e117b20f1ce81 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -19,19 +19,15 @@ #include -#include #include -#include #include #include #include #include -#include #include #include "common/status.h" #include "olap/memtable_memory_limiter.h" -#include "olap/olap_define.h" #include "olap/options.h" #include "olap/rowset/segment_v2/inverted_index_writer.h" #include "olap/tablet_fwd.h" @@ -52,6 +48,7 @@ class BlockedTaskScheduler; struct RuntimeFilterTimerQueue; } // namespace pipeline class WorkloadGroupMgr; +struct WriteCooldownMetaExecutors; namespace io { class FileCacheFactory; } // namespace io @@ -233,6 +230,9 @@ class ExecEnv { MemTableMemoryLimiter* memtable_memory_limiter() { return _memtable_memory_limiter.get(); } WalManager* wal_mgr() { return _wal_manager.get(); } DNSCache* dns_cache() { return _dns_cache; } + WriteCooldownMetaExecutors* write_cooldown_meta_executors() { + return _write_cooldown_meta_executors.get(); + } #ifdef BE_TEST void set_tmp_file_dir(std::unique_ptr tmp_file_dirs) { @@ -263,6 +263,7 @@ class ExecEnv { void set_dummy_lru_cache(std::shared_ptr dummy_lru_cache) { this->_dummy_lru_cache = dummy_lru_cache; } + void set_write_cooldown_meta_executors(); #endif LoadStreamMapPool* load_stream_map_pool() { return _load_stream_map_pool.get(); } @@ -396,6 +397,7 @@ class ExecEnv { std::unique_ptr _delta_writer_v2_pool; std::shared_ptr _wal_manager; DNSCache* _dns_cache = nullptr; + std::unique_ptr _write_cooldown_meta_executors; std::mutex _frontends_lock; // ip:brpc_port -> frontend_indo diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index d2e7e47f9e399f..309b6db1b3f98d 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -26,12 +26,9 @@ #include #include #include -#include #include #include #include -#include -#include #include #include "common/config.h" @@ -39,7 +36,6 @@ #include "common/status.h" #include "io/cache/block/block_file_cache_factory.h" #include "io/fs/file_meta_cache.h" -#include "io/fs/s3_file_bufferpool.h" #include "olap/memtable_memory_limiter.h" #include "olap/olap_define.h" #include "olap/options.h" @@ -234,6 +230,7 @@ Status ExecEnv::_init(const std::vector& store_paths, _delta_writer_v2_pool = std::make_unique(); _wal_manager = WalManager::create_shared(this, config::group_commit_wal_path); _dns_cache = new DNSCache(); + _write_cooldown_meta_executors = std::make_unique(); _spill_stream_mgr = new vectorized::SpillStreamManager(spill_store_paths); _backend_client_cache->init_metrics("backend"); @@ -567,8 +564,11 @@ void ExecEnv::destroy() { _memtable_memory_limiter.reset(); _delta_writer_v2_pool.reset(); _load_stream_map_pool.reset(); + SAFE_STOP(_storage_engine); + SAFE_STOP(_write_cooldown_meta_executors); SAFE_STOP(_spill_stream_mgr); + SAFE_SHUTDOWN(_buffered_reader_prefetch_thread_pool); SAFE_SHUTDOWN(_s3_file_upload_thread_pool); SAFE_SHUTDOWN(_join_node_thread_pool); @@ -625,6 +625,7 @@ void ExecEnv::destroy() { _buffered_reader_prefetch_thread_pool.reset(nullptr); _s3_file_upload_thread_pool.reset(nullptr); _send_batch_thread_pool.reset(nullptr); + _write_cooldown_meta_executors.reset(nullptr); SAFE_DELETE(_broker_client_cache); SAFE_DELETE(_frontend_client_cache); diff --git a/be/src/vec/columns/column_const.cpp b/be/src/vec/columns/column_const.cpp index 096ec91ecea6ef..6789de4e6a238f 100644 --- a/be/src/vec/columns/column_const.cpp +++ b/be/src/vec/columns/column_const.cpp @@ -156,7 +156,7 @@ ColumnPtr ColumnConst::index(const IColumn& indexes, size_t limit) const { } std::pair check_column_const_set_readability(const IColumn& column, - const size_t row_num) noexcept { + size_t row_num) noexcept { std::pair result; if (is_column_const(column)) { result.first = static_cast(column).get_data_column_ptr(); diff --git a/be/src/vec/columns/column_const.h b/be/src/vec/columns/column_const.h index 016e4257402ee0..66db2ed54f0ca1 100644 --- a/be/src/vec/columns/column_const.h +++ b/be/src/vec/columns/column_const.h @@ -21,11 +21,10 @@ #pragma once #include -#include #include -#include #include +#include #include #include #include @@ -48,14 +47,43 @@ class SipHash; -namespace doris { -namespace vectorized { +namespace doris::vectorized { + class Arena; class Block; -} // namespace vectorized -} // namespace doris -namespace doris::vectorized { +/* + * @return first : pointer to column itself if it's not ColumnConst, else to column's data column. + * second : zero if column is ColumnConst, else itself. +*/ +std::pair check_column_const_set_readability(const IColumn& column, + size_t row_num) noexcept; + +/* + * @warning use this function sometimes cause performance problem in GCC. +*/ +template + requires std::is_integral_v +T index_check_const(T arg, bool constancy) noexcept { + return constancy ? 0 : arg; +} + +/* + * @return first : data_column_ptr for ColumnConst, itself otherwise. + * second : whether it's ColumnConst. +*/ +std::pair unpack_if_const(const ColumnPtr&) noexcept; + +/* + * For the functions that some columns of arguments are almost but not completely always const, we use this function to preprocessing its parameter columns + * (which are not data columns). When we have two or more columns which only provide parameter, use this to deal with corner case. So you can specialize you + * implementations for all const or all parameters const, without considering some of parameters are const. + + * Do the transformation only for the columns whose arg_indexes in parameters. +*/ +void default_preprocess_parameter_columns(ColumnPtr* columns, const bool* col_const, + const std::initializer_list& parameters, + Block& block, const ColumnNumbers& arg_indexes) noexcept; /** ColumnConst contains another column with single element, * but looks like a column with arbitrary amount of same elements. @@ -142,7 +170,7 @@ class ColumnConst final : public COWHelper { } const char* deserialize_and_insert_from_arena(const char* pos) override { - auto res = data->deserialize_and_insert_from_arena(pos); + const auto* res = data->deserialize_and_insert_from_arena(pos); data->pop_back(1); ++s; return res; @@ -204,8 +232,9 @@ class ColumnConst final : public COWHelper { int compare_at(size_t, size_t, const IColumn& rhs, int nan_direction_hint) const override { auto rhs_const_column = assert_cast(rhs); - auto* this_nullable = check_and_get_column(data.get()); - auto* rhs_nullable = check_and_get_column(rhs_const_column.data.get()); + const auto* this_nullable = check_and_get_column(data.get()); + const auto* rhs_nullable = + check_and_get_column(rhs_const_column.data.get()); if (this_nullable && rhs_nullable) { return data->compare_at(0, 0, *rhs_const_column.data, nan_direction_hint); } else if (this_nullable) { @@ -228,8 +257,9 @@ class ColumnConst final : public COWHelper { void for_each_subcolumn(ColumnCallback callback) override { callback(data); } bool structure_equals(const IColumn& rhs) const override { - if (auto rhs_concrete = typeid_cast(&rhs)) + if (const auto* rhs_concrete = typeid_cast(&rhs)) { return data->structure_equals(*rhs_concrete->data); + } return false; } @@ -264,37 +294,4 @@ class ColumnConst final : public COWHelper { LOG(FATAL) << "should not call the method in column const"; } }; - -/* - * @return first : pointer to column itself if it's not ColumnConst, else to column's data column. - * second : zero if column is ColumnConst, else itself. -*/ -std::pair check_column_const_set_readability(const IColumn& column, - const size_t row_num) noexcept; - -/* - * @warning use this function sometimes cause performance problem in GCC. -*/ -template - requires std::is_integral_v -T index_check_const(T arg, bool constancy) noexcept { - return constancy ? 0 : arg; -} - -/* - * @return first : data_column_ptr for ColumnConst, itself otherwise. - * second : whether it's ColumnConst. -*/ -std::pair unpack_if_const(const ColumnPtr&) noexcept; - -/* - * For the functions that some columns of arguments are almost but not completely always const, we use this function to preprocessing its parameter columns - * (which are not data columns). When we have two or more columns which only provide parameter, use this to deal with corner case. So you can specialize you - * implementations for all const or all parameters const, without considering some of parameters are const. - - * Do the transformation only for the columns whose arg_indexes in parameters. -*/ -void default_preprocess_parameter_columns(ColumnPtr* columns, const bool* col_const, - const std::initializer_list& parameters, - Block& block, const ColumnNumbers& arg_indexes) noexcept; } // namespace doris::vectorized diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h index 462b28d3a7da0f..56dbc619a39212 100644 --- a/be/src/vec/columns/column_nullable.h +++ b/be/src/vec/columns/column_nullable.h @@ -89,7 +89,7 @@ class ColumnNullable final : public COWHelper { std::string get_name() const override { return "Nullable(" + nested_column->get_name() + ")"; } MutableColumnPtr clone_resized(size_t size) const override; size_t size() const override { return assert_cast(*null_map).size(); } - bool is_null_at(size_t n) const override { + PURE bool is_null_at(size_t n) const override { return assert_cast(*null_map).get_data()[n] != 0; } bool is_default_at(size_t n) const override { return is_null_at(n); } diff --git a/be/src/vec/common/assert_cast.h b/be/src/vec/common/assert_cast.h index 879f27ed8d81d3..6d8765befa203f 100644 --- a/be/src/vec/common/assert_cast.h +++ b/be/src/vec/common/assert_cast.h @@ -20,13 +20,10 @@ #pragma once -#include #include -#include #include #include "common/logging.h" -#include "fmt/format.h" #include "vec/common/demangle.h" /** Perform static_cast in release build. @@ -34,11 +31,13 @@ * The exact match of the type is checked. That is, cast to the ancestor will be unsuccessful. */ template -To assert_cast(From&& from) { +PURE To assert_cast(From&& from) { #ifndef NDEBUG try { if constexpr (std::is_pointer_v) { - if (typeid(*from) == typeid(std::remove_pointer_t)) return static_cast(from); + if (typeid(*from) == typeid(std::remove_pointer_t)) { + return static_cast(from); + } if constexpr (std::is_pointer_v>) { if (auto ptr = dynamic_cast(from); ptr != nullptr) { return ptr; @@ -48,7 +47,9 @@ To assert_cast(From&& from) { demangle(typeid(To).name())); } } else { - if (typeid(from) == typeid(To)) return static_cast(from); + if (typeid(from) == typeid(To)) { + return static_cast(from); + } } } catch (const std::exception& e) { LOG(FATAL) << "assert cast err:" << e.what(); diff --git a/be/src/vec/sink/vrow_distribution.cpp b/be/src/vec/sink/vrow_distribution.cpp index 052194a15a0e01..3bef7d0117ec51 100644 --- a/be/src/vec/sink/vrow_distribution.cpp +++ b/be/src/vec/sink/vrow_distribution.cpp @@ -21,7 +21,9 @@ #include #include +#include #include +#include #include "common/status.h" #include "runtime/client_cache.h" @@ -33,6 +35,7 @@ #include "vec/columns/column_const.h" #include "vec/columns/column_nullable.h" #include "vec/columns/column_vector.h" +#include "vec/common/assert_cast.h" #include "vec/data_types/data_type.h" #include "vec/sink/writer/vtablet_writer.h" @@ -43,17 +46,24 @@ VRowDistribution::_get_partition_function() { return {_vpartition->get_part_func_ctx(), _vpartition->get_partition_function()}; } -Status VRowDistribution::_save_missing_values(std::vector>& col_strs, - int col_size, Block* block, - std::vector filter) { +Status VRowDistribution::_save_missing_values( + std::vector>& col_strs, // non-const ref for move + int col_size, Block* block, std::vector filter, + const std::vector& col_null_maps) { // de-duplication for new partitions but save all rows. _batching_block->add_rows(block, filter); - std::vector cur_row_values; + std::vector cur_row_values; for (int row = 0; row < col_strs[0].size(); ++row) { cur_row_values.clear(); for (int col = 0; col < col_size; ++col) { - TStringLiteral node; - node.value = std::move(col_strs[col][row]); + TNullableStringLiteral node; + const auto* null_map = col_null_maps[col]; // null map for this col + node.__set_is_null((null_map && (*null_map)[filter[row]]) + ? true + : node.is_null); // if not, dont change(default false) + if (!node.is_null) { + node.__set_value(col_strs[col][row]); + } cur_row_values.push_back(node); } //For duplicate cur_values, they will be filtered in FE @@ -83,7 +93,7 @@ Status VRowDistribution::automatic_create_partition() { request.__set_table_id(_vpartition->table_id()); request.__set_partitionValues(_partitions_need_create); - VLOG(1) << "automatic partition rpc begin request " << request; + VLOG_NOTICE << "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( @@ -94,7 +104,7 @@ Status VRowDistribution::automatic_create_partition() { time_out)); Status status(Status::create(result.status)); - VLOG(1) << "automatic partition rpc end response " << result; + VLOG_NOTICE << "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)); @@ -104,6 +114,73 @@ Status VRowDistribution::automatic_create_partition() { return status; } +// for reuse the same create callback of create-partition +static TCreatePartitionResult cast_as_create_result(TReplacePartitionResult& arg) { + TCreatePartitionResult result; + result.status = arg.status; + result.nodes = std::move(arg.nodes); + result.partitions = std::move(arg.partitions); + result.tablets = std::move(arg.tablets); + return result; +} + +// use _partitions and replace them +Status VRowDistribution::_replace_overwriting_partition() { + SCOPED_TIMER(_add_partition_request_timer); + TReplacePartitionRequest request; + TReplacePartitionResult result; + request.__set_overwrite_group_id(_vpartition->get_overwrite_group_id()); + request.__set_db_id(_vpartition->db_id()); + request.__set_table_id(_vpartition->table_id()); + + // only request for partitions not recorded for replacement + std::set id_deduper; + for (const auto* part : _partitions) { + if (part == nullptr) [[unlikely]] { + return Status::InternalError( + "Cannot found origin partitions in auto detect overwriting, stop processing"); + } + if (_new_partition_ids.contains(part->id)) { + // this is a new partition. dont replace again. + } else { + // request for replacement + id_deduper.insert(part->id); + } + } + if (id_deduper.empty()) { + return Status::OK(); // no need to request + } + // de-duplicate. there's no check in FE + std::vector request_part_ids(id_deduper.begin(), id_deduper.end()); + request.__set_partition_ids(request_part_ids); + + VLOG_NOTICE << "auto detect replace partition 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->replacePartition(result, request); + }, + time_out)); + + Status status(Status::create(result.status)); + VLOG_NOTICE << "auto detect replace partition result: " << result; + if (result.status.status_code == TStatusCode::OK) { + // record new partitions + for (const auto& part : result.partitions) { + _new_partition_ids.insert(part.id); + } + // replace data in _partitions + RETURN_IF_ERROR(_vpartition->replace_partitions(request_part_ids, result.partitions)); + // reuse the function as the args' structure are same. it add nodes/locations and incremental_open + auto result_as_create = cast_as_create_result(result); + RETURN_IF_ERROR(_create_partition_callback(_caller, &result_as_create)); + } + + return status; +} + void VRowDistribution::_get_tablet_ids(vectorized::Block* block, int32_t index_idx, std::vector& tablet_ids) { tablet_ids.reserve(block->rows()); @@ -222,7 +299,6 @@ Status VRowDistribution::_generate_rows_distribution_for_auto_partition( auto num_rows = block->rows(); std::vector partition_keys = _vpartition->get_partition_keys(); - //TODO: use loop to create missing_vals for multi column. auto partition_col = block->get_by_position(partition_keys[0]); _missing_map.clear(); _missing_map.reserve(partition_col.column->size()); @@ -242,29 +318,34 @@ Status VRowDistribution::_generate_rows_distribution_for_auto_partition( if (!_missing_map.empty()) { // for missing partition keys, calc the missing partition and save in _partitions_need_create - auto [part_ctxs, part_funcs] = _get_partition_function(); - auto funcs_size = part_funcs.size(); + auto [part_ctxs, part_exprs] = _get_partition_function(); + auto part_col_num = part_exprs.size(); + // the two vectors are in column-first-order std::vector> col_strs; - col_strs.resize(funcs_size); - - for (int i = 0; i < funcs_size; ++i) { - auto return_type = part_funcs[i]->data_type(); - // expose the data column - vectorized::ColumnPtr range_left_col = - block->get_by_position(partition_cols_idx[i]).column; - if (const 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(); + std::vector col_null_maps; + col_strs.resize(part_col_num); + col_null_maps.reserve(part_col_num); + + for (int i = 0; i < part_col_num; ++i) { + auto return_type = part_exprs[i]->data_type(); + // expose the data column. the return type would be nullable + const auto& [range_left_col, col_const] = + unpack_if_const(block->get_by_position(partition_cols_idx[i]).column); + if (range_left_col->is_nullable()) { + col_null_maps.push_back(&(assert_cast(range_left_col.get()) + ->get_null_map_data())); + } else { + col_null_maps.push_back(nullptr); } for (auto row : _missing_map) { - col_strs[i].push_back(return_type->to_string(*range_left_col, row)); + col_strs[i].push_back( + return_type->to_string(*range_left_col, index_check_const(row, col_const))); } } // calc the end value and save them. in the end of sending, we will create partitions for them and deal them. - RETURN_IF_ERROR(_save_missing_values(col_strs, funcs_size, block, _missing_map)); + RETURN_IF_ERROR( + _save_missing_values(col_strs, part_col_num, block, _missing_map, col_null_maps)); size_t new_bt_rows = _batching_block->rows(); size_t new_bt_bytes = _batching_block->bytes(); @@ -279,6 +360,29 @@ Status VRowDistribution::_generate_rows_distribution_for_auto_partition( return Status::OK(); } +Status VRowDistribution::_generate_rows_distribution_for_auto_overwrite( + vectorized::Block* block, bool has_filtered_rows, + std::vector& row_part_tablet_ids) { + auto num_rows = block->rows(); + + bool stop_processing = false; + RETURN_IF_ERROR(_tablet_finder->find_tablets(_state, block, num_rows, _partitions, + _tablet_indexes, stop_processing, _skip)); + RETURN_IF_ERROR(_replace_overwriting_partition()); + + // regenerate locations for new partitions & tablets + _reset_find_tablets(num_rows); + RETURN_IF_ERROR(_tablet_finder->find_tablets(_state, block, num_rows, _partitions, + _tablet_indexes, stop_processing, _skip)); + if (has_filtered_rows) { + for (int i = 0; i < num_rows; i++) { + _skip[i] = _skip[i] || _block_convertor->filter_map()[i]; + } + } + RETURN_IF_ERROR(_filter_block(block, row_part_tablet_ids)); + return Status::OK(); +} + void VRowDistribution::_reset_row_part_tablet_ids( std::vector& row_part_tablet_ids, int64_t rows) { row_part_tablet_ids.resize(_schema->indexes().size()); @@ -316,12 +420,7 @@ Status VRowDistribution::generate_rows_distribution( } auto num_rows = block->rows(); - _tablet_finder->filter_bitmap().Reset(num_rows); - - //reuse vars for find_tablets - _partitions.assign(num_rows, nullptr); - _skip.assign(num_rows, false); - _tablet_indexes.assign(num_rows, 0); + _reset_find_tablets(num_rows); // if there's projection of partition calc, we need to calc it first. auto [part_ctxs, part_funcs] = _get_partition_function(); @@ -331,6 +430,7 @@ Status VRowDistribution::generate_rows_distribution( auto func_size = part_funcs.size(); for (int i = 0; i < func_size; ++i) { int result_idx = -1; + // we just calc left range here. leave right to FE to avoid dup calc. RETURN_IF_ERROR(part_funcs[i]->execute(part_ctxs[i].get(), block.get(), &result_idx)); VLOG_DEBUG << "Partition-calculated block:" << block->dump_data(0, 1); @@ -343,7 +443,11 @@ Status VRowDistribution::generate_rows_distribution( _vpartition->set_transformed_slots(partition_cols_idx); } - if (_vpartition->is_auto_partition() && !_deal_batched) { + if (_vpartition->is_auto_detect_overwrite()) { + // when overwrite, no auto create partition allowed. + RETURN_IF_ERROR(_generate_rows_distribution_for_auto_overwrite( + block.get(), has_filtered_rows, row_part_tablet_ids)); + } else if (_vpartition->is_auto_partition() && !_deal_batched) { RETURN_IF_ERROR(_generate_rows_distribution_for_auto_partition( block.get(), partition_cols_idx, has_filtered_rows, row_part_tablet_ids, rows_stat_val)); @@ -357,4 +461,12 @@ Status VRowDistribution::generate_rows_distribution( return Status::OK(); } +// reuse vars for find_tablets +void VRowDistribution::_reset_find_tablets(int64_t rows) { + _tablet_finder->filter_bitmap().Reset(rows); + _partitions.assign(rows, nullptr); + _skip.assign(rows, false); + _tablet_indexes.assign(rows, 0); +} + } // namespace doris::vectorized diff --git a/be/src/vec/sink/vrow_distribution.h b/be/src/vec/sink/vrow_distribution.h index 12acda73ed2033..5267b488400b8f 100644 --- a/be/src/vec/sink/vrow_distribution.h +++ b/be/src/vec/sink/vrow_distribution.h @@ -18,23 +18,21 @@ #pragma once // IWYU pragma: no_include +#include #include #include #include #include #include -#include #include #include "common/status.h" #include "exec/tablet_info.h" #include "runtime/runtime_state.h" -#include "runtime/types.h" #include "util/runtime_profile.h" -#include "util/stopwatch.hpp" #include "vec/core/block.h" -#include "vec/data_types/data_type.h" +#include "vec/exprs/vexpr_context.h" #include "vec/exprs/vexpr_fwd.h" #include "vec/sink/vtablet_block_convertor.h" #include "vec/sink/vtablet_finder.h" @@ -50,6 +48,15 @@ class RowPartTabletIds { std::vector row_ids; std::vector partition_ids; std::vector tablet_ids; + + std::string debug_string() const { + std::string value; + value.reserve(row_ids.size() * 15); + for (int i = 0; i < row_ids.size(); i++) { + value.append(fmt::format("[{}, {}, {}]", row_ids[i], partition_ids[i], tablet_ids[i])); + } + return value; + } }; // void* for caller @@ -130,7 +137,8 @@ class VRowDistribution { std::pair _get_partition_function(); Status _save_missing_values(std::vector>& col_strs, int col_size, - Block* block, std::vector filter); + Block* block, std::vector filter, + const std::vector& col_null_maps); void _get_tablet_ids(vectorized::Block* block, int32_t index_idx, std::vector& tablet_ids); @@ -153,14 +161,20 @@ class VRowDistribution { vectorized::Block* block, bool has_filtered_rows, std::vector& row_part_tablet_ids); + Status _generate_rows_distribution_for_auto_overwrite( + vectorized::Block* block, bool has_filtered_rows, + std::vector& row_part_tablet_ids); + Status _replace_overwriting_partition(); + void _reset_row_part_tablet_ids(std::vector& row_part_tablet_ids, int64_t rows); + void _reset_find_tablets(int64_t rows); RuntimeState* _state = nullptr; int _batch_size = 0; // for auto partitions - std::vector> _partitions_need_create; + std::vector> _partitions_need_create; public: std::unique_ptr _batching_block; @@ -177,16 +191,19 @@ class VRowDistribution { OlapTableLocationParam* _location = nullptr; // int64_t _number_output_rows = 0; const VExprContextSPtrs* _vec_output_expr_ctxs = nullptr; + // generally it's writer's on_partitions_created CreatePartitionCallback _create_partition_callback = nullptr; void* _caller = nullptr; std::shared_ptr _schema; - // reuse for find_tablet. + // reuse for find_tablet. save partitions found by find_tablets std::vector _partitions; std::vector _skip; std::vector _tablet_indexes; std::vector _tablet_ids; std::vector _missing_map; // indice of missing values in partition_col + // for auto detect overwrite partition + std::set _new_partition_ids; // if contains, not to replace it again. }; } // namespace doris::vectorized diff --git a/be/src/vec/sink/vtablet_finder.cpp b/be/src/vec/sink/vtablet_finder.cpp index 865a3066d62f8c..2e0d278fa4fe01 100644 --- a/be/src/vec/sink/vtablet_finder.cpp +++ b/be/src/vec/sink/vtablet_finder.cpp @@ -22,22 +22,14 @@ #include #include -#include #include -#include #include #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::vectorized { Status OlapTabletFinder::find_tablets(RuntimeState* state, Block* block, int rows, diff --git a/be/src/vec/sink/vtablet_finder.h b/be/src/vec/sink/vtablet_finder.h index 537e38d86c3d34..24f8e357e28976 100644 --- a/be/src/vec/sink/vtablet_finder.h +++ b/be/src/vec/sink/vtablet_finder.h @@ -17,8 +17,8 @@ #pragma once +#include #include -#include #include "common/status.h" #include "exec/tablet_info.h" @@ -53,6 +53,7 @@ class OlapTabletFinder { bool is_single_tablet() { return _partition_to_tablet_map.size() == 1; } + // all partitions for multi find-processes of its relative writer. const vectorized::flat_hash_set& partition_ids() { return _partition_ids; } int64_t num_filtered_rows() const { return _num_filtered_rows; } diff --git a/be/src/vec/sink/writer/vtablet_writer.cpp b/be/src/vec/sink/writer/vtablet_writer.cpp index cf6dc0b70ed588..b252451950ce5f 100644 --- a/be/src/vec/sink/writer/vtablet_writer.cpp +++ b/be/src/vec/sink/writer/vtablet_writer.cpp @@ -672,9 +672,8 @@ void VNodeChannel::try_send_pending_block(RuntimeState* state) { request->add_partition_ids(pid); } - request->set_write_single_replica(false); + request->set_write_single_replica(_parent->_write_single_replica); if (_parent->_write_single_replica) { - request->set_write_single_replica(true); for (auto& _slave_tablet_node : _slave_tablet_nodes) { PSlaveTabletNodes slave_tablet_nodes; for (auto node_id : _slave_tablet_node.second) { diff --git a/be/src/vec/sink/writer/vtablet_writer_v2.cpp b/be/src/vec/sink/writer/vtablet_writer_v2.cpp index a9ac012bf71174..af1caefa43d15c 100644 --- a/be/src/vec/sink/writer/vtablet_writer_v2.cpp +++ b/be/src/vec/sink/writer/vtablet_writer_v2.cpp @@ -24,13 +24,14 @@ #include #include -#include +#include #include #include #include #include #include "common/compiler_util.h" // IWYU pragma: keep +#include "common/logging.h" #include "common/object_pool.h" #include "common/signal_handler.h" #include "common/status.h" @@ -40,18 +41,14 @@ #include "runtime/exec_env.h" #include "runtime/runtime_state.h" #include "runtime/thread_context.h" -#include "service/brpc.h" -#include "util/brpc_client_cache.h" #include "util/debug_points.h" #include "util/defer_op.h" #include "util/doris_metrics.h" -#include "util/threadpool.h" -#include "util/thrift_util.h" #include "util/uid_util.h" #include "vec/core/block.h" #include "vec/sink/delta_writer_v2_pool.h" #include "vec/sink/load_stream_map_pool.h" -#include "vec/sink/load_stream_stub.h" +#include "vec/sink/load_stream_stub.h" // IWYU pragma: keep #include "vec/sink/vtablet_block_convertor.h" #include "vec/sink/vtablet_finder.h" @@ -107,6 +104,8 @@ Status VTabletWriterV2::_incremental_open_streams( } _indexes_from_node[node].emplace_back(tablet); known_indexes.insert(index.index_id); + VLOG_DEBUG << "incremental open stream (" << partition->id << ", " << tablet_id + << ")"; } } } @@ -205,7 +204,7 @@ Status VTabletWriterV2::_init(RuntimeState* state, RuntimeProfile* profile) { "output_tuple_slot_num {} should be equal to output_expr_num {}", _output_tuple_desc->slots().size() + 1, _vec_output_expr_ctxs.size()); }); - if (_vec_output_expr_ctxs.size() > 0 && + if (!_vec_output_expr_ctxs.empty() && _output_tuple_desc->slots().size() != _vec_output_expr_ctxs.size()) { LOG(WARNING) << "output tuple slot num should be equal to num of output exprs, " << "output_tuple_slot_num " << _output_tuple_desc->slots().size() @@ -296,7 +295,7 @@ Status VTabletWriterV2::_build_tablet_node_mapping() { for (const auto& partition : _vpartition->get_partitions()) { for (const auto& index : partition->indexes) { for (const auto& tablet_id : index.tablets) { - auto tablet_location = _location->find_tablet(tablet_id); + auto* tablet_location = _location->find_tablet(tablet_id); DBUG_EXECUTE_IF("VTabletWriterV2._build_tablet_node_mapping.tablet_location_null", { tablet_location = nullptr; }); if (tablet_location == nullptr) { @@ -357,6 +356,7 @@ Status VTabletWriterV2::_select_streams(int64_t tablet_id, int64_t partition_id, tablet.set_partition_id(partition_id); tablet.set_index_id(index_id); tablet.set_tablet_id(tablet_id); + VLOG_DEBUG << fmt::format("_select_streams P{} I{} T{}", partition_id, index_id, tablet_id); _tablets_for_node[node_id].emplace(tablet_id, tablet); streams.emplace_back(_load_stream_map->at(node_id)->at(_stream_index)); RETURN_IF_ERROR(streams[0]->wait_for_schema(partition_id, index_id, tablet_id)); diff --git a/be/test/olap/tablet_cooldown_test.cpp b/be/test/olap/tablet_cooldown_test.cpp index 984afcc90dcdc4..ba0d8b87d05ff1 100644 --- a/be/test/olap/tablet_cooldown_test.cpp +++ b/be/test/olap/tablet_cooldown_test.cpp @@ -228,6 +228,7 @@ class TabletCooldownTest : public testing::Test { st = k_engine->open(); EXPECT_TRUE(st.ok()) << st.to_string(); ExecEnv* exec_env = doris::ExecEnv::GetInstance(); + exec_env->set_write_cooldown_meta_executors(); // default cons exec_env->set_memtable_memory_limiter(new MemTableMemoryLimiter()); exec_env->set_storage_engine(k_engine.get()); } diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Manipulation-Statements/Manipulation/INSERT-OVERWRITE.md b/docs/en/docs/sql-manual/sql-reference/Data-Manipulation-Statements/Manipulation/INSERT-OVERWRITE.md index f94e37f5e01ca2..7293785ba536ef 100644 --- a/docs/en/docs/sql-manual/sql-reference/Data-Manipulation-Statements/Manipulation/INSERT-OVERWRITE.md +++ b/docs/en/docs/sql-manual/sql-reference/Data-Manipulation-Statements/Manipulation/INSERT-OVERWRITE.md @@ -33,11 +33,11 @@ INSERT OVERWRITE ### Description -The function of this statement is to overwrite a table or a partition of a table +The function of this statement is to overwrite a table or some partitions of a table ```sql INSERT OVERWRITE table table_name - [ PARTITION (p1, ...) ] + [ PARTITION (p1, ... | *) ] [ WITH LABEL label] [ (column [, ...]) ] [ [ hint [, ...] ] ] @@ -48,7 +48,10 @@ INSERT OVERWRITE table table_name > table_name: the destination table to overwrite. This table must exist. It can be of the form `db_name.table_name` > -> partitions: the table partition that needs to be overwritten must be one of the existing partitions in `table_name` separated by a comma +> partitions: the table partitions that needs to be overwritten. The following two formats are supported +> +>> 1. partition names. must be one of the existing partitions in `table_name` separated by a comma +>> 2. asterisk(*)。Enable [auto-detect-partition](#overwrite-auto-detect-partition). The write operation will automatically detect the partitions involved in the data and overwrite those partitions. > > label: specify a label for the Insert task > @@ -69,7 +72,7 @@ INSERT OVERWRITE table table_name Notice: 1. In the current version, the session variable `enable_insert_strict` is set to `true` by default. If some data that does not conform to the format of the target table is filtered out during the execution of the `INSERT OVERWRITE` statement, such as when overwriting a partition and not all partition conditions are satisfied, overwriting the target table will fail. -2. If the target table of the INSERT OVERWRITE is an [AUTO-PARTITION-table](../../../../advanced/partition/auto-partition), then new partitions can be created if PARTITION is not specified (that is, rewrite the whole table). If PARTITION for overwrite is specified, then the AUTO PARTITION table behaves as if it were a normal partitioned table during this process, and data that does not satisfy the existing partition conditions is filtered instead of creating a new partition. +2. If the target table of the INSERT OVERWRITE is an [AUTO-PARTITION-table](../../../../advanced/partition/auto-partition), then new partitions can be created if PARTITION is not specified (that is, rewrite the whole table). If PARTITION for overwrite is specified(Includes automatic detection and overwriting of partitions through the `partition(*)` syntax), then the AUTO PARTITION table behaves as if it were a normal partitioned table during this process, and data that does not satisfy the existing partition conditions is filtered instead of creating a new partition. 3. The `INSERT OVERWRITE` statement first creates a new table, inserts the data to be overwritten into the new table, and then atomically replaces the old table with the new table and modifies its name. Therefore, during the process of overwriting the table, the data in the old table can still be accessed normally until the overwriting is completed. ### Example @@ -138,6 +141,13 @@ PROPERTIES ( #### Overwrite Table Partition +When using INSERT OVERWRITE to rewrite partitions, we actually encapsulate the following three steps into a single transaction and execute it. If it fails halfway through, the operations that have been performed will be rolled back: +1. Assuming that partition `p1` is specified to be rewritten, first create an empty temporary partition `pTMP` with the same structure as the target partition to be rewritten. +2. Write data to `pTMP`. +3. replace `p1` with the `pTMP` atom + +The following is examples: + 1. Overwrite partitions `P1` and `P2` of the `test` table using the form of `VALUES`. ```sql @@ -175,6 +185,56 @@ PROPERTIES ( INSERT OVERWRITE table test PARTITION(p1,p2) WITH LABEL `label4` (c1, c2) SELECT * from test2; ``` + +#### Overwrite Auto Detect Partition + +When the PARTITION clause specified by the INSERT OVERWRITE command is `PARTITION(*)`, this overwrite will automatically detect the partition where the data is located. Example: + +```sql +mysql> create table test( + -> k0 int null + -> ) + -> partition by range (k0) + -> ( + -> PARTITION p10 values less than (10), + -> PARTITION p100 values less than (100), + -> PARTITION pMAX values less than (maxvalue) + -> ) + -> DISTRIBUTED BY HASH(`k0`) BUCKETS 1 + -> properties("replication_num" = "1"); +Query OK, 0 rows affected (0.11 sec) + +mysql> insert into test values (1), (2), (15), (100), (200); +Query OK, 5 rows affected (0.29 sec) + +mysql> select * from test order by k0; ++------+ +| k0 | ++------+ +| 1 | +| 2 | +| 15 | +| 100 | +| 200 | ++------+ +5 rows in set (0.23 sec) + +mysql> insert overwrite table test partition(*) values (3), (1234); +Query OK, 2 rows affected (0.24 sec) + +mysql> select * from test order by k0; ++------+ +| k0 | ++------+ +| 3 | +| 15 | +| 1234 | ++------+ +3 rows in set (0.20 sec) +``` + +As you can see, all data in partitions `p10` and `pMAX`, where data 3 and 1234 are located, are overwritten, while partition `p100` remains unchanged. This operation can be interpreted as syntactic sugar for specifying a specific partition to be overwritten by the PARTITION clause during an INSERT OVERWRITE operation, which is implemented in the same way as [specify a partition to overwrite](#overwrite-table-partition). The `PARTITION(*)` syntax eliminates the need to manually fill in all the partition names when overwriting a large number of partitions. + ### Keywords - INSERT OVERWRITE, OVERWRITE + INSERT OVERWRITE, OVERWRITE, AUTO DETECT diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Manipulation-Statements/Manipulation/INSERT-OVERWRITE.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Manipulation-Statements/Manipulation/INSERT-OVERWRITE.md index c506029155b19e..587a981c24b1a1 100644 --- a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Manipulation-Statements/Manipulation/INSERT-OVERWRITE.md +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Manipulation-Statements/Manipulation/INSERT-OVERWRITE.md @@ -33,11 +33,11 @@ INSERT OVERWRITE ### Description -该语句的功能是重写表或表的某个分区 +该语句的功能是重写表或表的某些分区 ```sql INSERT OVERWRITE table table_name - [ PARTITION (p1, ...) ] + [ PARTITION (p1, ... | *) ] [ WITH LABEL label] [ (column [, ...]) ] [ [ hint [, ...] ] ] @@ -48,7 +48,10 @@ INSERT OVERWRITE table table_name > table_name: 需要重写的目的表。这个表必须存在。可以是 `db_name.table_name` 形式 > -> partitions: 需要重写的表分区,必须是 `table_name` 中存在的分区,多个分区名称用逗号分隔 +> partitions: 需要重写的目标分区,支持两种形式: +> +>> 1. 分区名。必须是 `table_name` 中存在的分区,多个分区名称用逗号分隔。 +>> 2. 星号(*)。开启[自动检测分区](#overwrite-auto-detect-partition)功能。写入操作将会自动检测数据所涉及的分区,并覆写这些分区。 > > label: 为 Insert 任务指定一个 label > @@ -69,7 +72,7 @@ INSERT OVERWRITE table table_name 注意: 1. 在当前版本中,会话变量 `enable_insert_strict` 默认为 `true`,如果执行 `INSERT OVERWRITE` 语句时,对于有不符合目标表格式的数据被过滤掉的话会重写目标表失败(比如重写分区时,不满足所有分区条件的数据会被过滤)。 -2. 如果INSERT OVERWRITE的目标表是[AUTO-PARTITION表](../../../../advanced/partition/auto-partition),若未指定PARTITION(重写整表),那么可以创建新的分区。如果指定了覆写的PARTITION,那么在此过程中,AUTO PARTITION表表现得如同普通分区表一样,不满足现有分区条件的数据将被过滤,而非创建新的分区。 +2. 如果INSERT OVERWRITE的目标表是[AUTO-PARTITION表](../../../../advanced/partition/auto-partition),若未指定PARTITION(重写整表),那么可以创建新的分区。如果指定了覆写的PARTITION(包括通过 `partition(*)` 语法自动检测并覆盖分区),那么在此过程中,AUTO PARTITION表表现得如同普通分区表一样,不满足现有分区条件的数据将被过滤,而非创建新的分区。 3. INSERT OVERWRITE语句会首先创建一个新表,将需要重写的数据插入到新表中,最后原子性的用新表替换旧表并修改名称。因此,在重写表的过程中,旧表中的数据在重写完毕之前仍然可以正常访问。 ### Example @@ -139,6 +142,13 @@ PROPERTIES ( #### Overwrite Table Partition +使用 INSERT OVERWRITE 重写分区时,实际我们是将如下三步操作封装为一个事务并执行,如果中途失败,已进行的操作将会回滚: +1. 假设指定重写分区 p1,首先创建一个与重写的目标分区结构相同的空临时分区 `pTMP` +2. 向 `pTMP` 中写入数据 +3. 使用 `pTMP` 原子替换 `p1` 分区 + +举例如下: + 1. VALUES的形式重写`test`表分区`P1`和`p2` ```sql @@ -176,7 +186,56 @@ PROPERTIES ( INSERT OVERWRITE table test PARTITION(p1,p2) WITH LABEL `label4` (c1, c2) SELECT * from test2; ``` +#### Overwrite Auto Detect Partition + +当 INSERT OVERWRITE 命令指定的 PARTITION 子句为 `PARTITION(*)` 时,此次覆写将会自动检测分区数据所在的分区。例如: + +```sql +mysql> create table test( + -> k0 int null + -> ) + -> partition by range (k0) + -> ( + -> PARTITION p10 values less than (10), + -> PARTITION p100 values less than (100), + -> PARTITION pMAX values less than (maxvalue) + -> ) + -> DISTRIBUTED BY HASH(`k0`) BUCKETS 1 + -> properties("replication_num" = "1"); +Query OK, 0 rows affected (0.11 sec) + +mysql> insert into test values (1), (2), (15), (100), (200); +Query OK, 5 rows affected (0.29 sec) + +mysql> select * from test order by k0; ++------+ +| k0 | ++------+ +| 1 | +| 2 | +| 15 | +| 100 | +| 200 | ++------+ +5 rows in set (0.23 sec) + +mysql> insert overwrite table test partition(*) values (3), (1234); +Query OK, 2 rows affected (0.24 sec) + +mysql> select * from test order by k0; ++------+ +| k0 | ++------+ +| 3 | +| 15 | +| 1234 | ++------+ +3 rows in set (0.20 sec) +``` + +可以看到,数据 3、1234 所在的分区 `p10` 和 `pMAX` 中的全部数据均被覆写,而 `p100` 分区未发生变化。该操作可以理解为 INSERT OVERWRITE 操作时通过 PARTITION 子句指定覆写特定分区的语法糖,它的实现原理与[指定重写特定分区](#overwrite-table-partition)相同。通过 `PARTITION(*)` 的语法,在覆写大量分区数据时我们可以免于手动填写全部分区名的繁琐。 + ### Keywords - INSERT OVERWRITE, OVERWRITE + INSERT OVERWRITE, OVERWRITE, AUTO DETECT 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 9f911d71fe1e0d..22fe3355fffc7f 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 @@ -124,9 +124,9 @@ constraint partitionSpec : TEMPORARY? (PARTITION | PARTITIONS) partitions=identifierList | TEMPORARY? PARTITION partition=errorCapturingIdentifier - // TODO: support analyze external table partition spec https://github.com/apache/doris/pull/24154 - // | PARTITIONS LEFT_PAREN ASTERISK RIGHT_PAREN - // | PARTITIONS WITH RECENT + | (PARTITION | PARTITIONS) LEFT_PAREN ASTERISK RIGHT_PAREN // for auto detect partition in overwriting + // TODO: support analyze external table partition spec https://github.com/apache/doris/pull/24154 + // | PARTITIONS WITH RECENT ; partitionTable diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index ee0f81951477dc..07f62881b41f9b 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -3411,6 +3411,10 @@ partition_value_list ::= {: RESULT = l; :} + | KW_NULL + {: + RESULT = Lists.newArrayList(new PartitionValue("", true)); + :} ; /* List */ @@ -3433,6 +3437,10 @@ partition_key_item_list ::= l.add(new PartitionValue(item)); RESULT = l; :} + | KW_NULL + {: + RESULT = Lists.newArrayList(new PartitionValue("", true)); + :} ; partition_key_list ::= @@ -3468,6 +3476,10 @@ partition_key_list ::= {: RESULT = Lists.newArrayList(PartitionValue.MAX_VALUE); :} + | KW_NULL + {: + RESULT = Lists.newArrayList(new PartitionValue("", true)); + :} ; fixed_partition_key_desc ::= @@ -6054,6 +6066,10 @@ partition_names ::= {: RESULT = new PartitionNames(true); :} + | KW_PARTITION LPAREN STAR RPAREN + {: + RESULT = new PartitionNames(true); + :} | KW_PARTITIONS KW_WITH KW_RECENT INTEGER_LITERAL:count {: RESULT = new PartitionNames(count); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyzeTblStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyzeTblStmt.java index c6c95da9de0ed8..08efb31d6631fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyzeTblStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyzeTblStmt.java @@ -255,11 +255,15 @@ public Set getPartitionNames() { return partitions; } - public boolean isAllPartitions() { + /** + * @return for OLAP table, only in overwrite situation, overwrite auto detect partition + * for External table, all partitions. + */ + public boolean isStarPartition() { if (partitionNames == null) { return false; } - return partitionNames.isAllPartitions(); + return partitionNames.isStar(); } public long getPartitionCount() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java index 125f1a56c9c057..a8148237fb7a52 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java @@ -26,7 +26,6 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.InvalidFormatException; import org.apache.doris.nereids.util.DateUtils; -import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; import org.apache.doris.thrift.TDateLiteral; import org.apache.doris.thrift.TExprNode; @@ -797,12 +796,9 @@ protected void toThrift(TExprNode msg) { try { checkValueValid(); } catch (AnalysisException e) { - if (ConnectContext.get() != null) { - ConnectContext.get().getState().reset(); - } - // If date value is invalid, set this to null - msg.node_type = TExprNodeType.NULL_LITERAL; - msg.setIsNullable(true); + // we must check before here. when we think we are ready to send thrift msg, + // the invalid value is not acceptable. we can't properly deal with it. + LOG.warn("meet invalid value when plan to translate " + toString() + " to thrift node"); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/InsertOverwriteTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/InsertOverwriteTableStmt.java index 17cca1cecc5e06..3b4e651cc02071 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/InsertOverwriteTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/InsertOverwriteTableStmt.java @@ -73,6 +73,13 @@ public List getPartitionNames() { return target.getPartitionNames().getPartitionNames(); } + /* + * auto detect which partitions to replace. enable by partition(*) grammer + */ + public boolean isAutoDetectPartition() { + return target.getPartitionNames().isStar(); + } + @Override public void analyze(Analyzer analyzer) throws UserException { target.getTblName().analyze(analyzer); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/LiteralExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/LiteralExpr.java index eb6fedb7b503eb..89e87699e7118a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/LiteralExpr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/LiteralExpr.java @@ -334,6 +334,7 @@ public boolean equals(Object obj) { @Override public boolean isNullable() { + // TODO: use base class's isNullLiteral() to replace this return this instanceof NullLiteral; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/MaxLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/MaxLiteral.java index b0d54eb5032cd7..57032f1683e20c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/MaxLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/MaxLiteral.java @@ -50,6 +50,7 @@ public int compareLiteral(LiteralExpr expr) { @Override protected void toThrift(TExprNode msg) { + // TODO: complete this type } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java index efe14a00cfa6e8..7a1ad0f786df2a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java @@ -162,6 +162,7 @@ public class NativeInsertStmt extends InsertStmt { boolean hasEmptyTargetColumns = false; private boolean allowAutoPartition = true; + private boolean withAutoDetectOverwrite = false; enum InsertType { NATIVE_INSERT("insert_"), @@ -316,6 +317,11 @@ public boolean isTransactionBegin() { return isTransactionBegin; } + public NativeInsertStmt withAutoDetectOverwrite() { + this.withAutoDetectOverwrite = true; + return this; + } + protected void preCheckAnalyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); 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 eb1bc7af747f9c..288d5a8457a814 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 @@ -201,10 +201,6 @@ public void analyze(List columnDefs, Map otherPropert throw new AnalysisException("Complex type column can't be partition column: " + columnDef.getType().toString()); } - // prohibit to create auto partition with null column anyhow - if (this.isAutoCreatePartitions && columnDef.isAllowNull()) { - throw new AnalysisException("The auto partition column must be NOT NULL"); - } if (!ConnectContext.get().getSessionVariable().isAllowPartitionColumnNullable() && columnDef.isAllowNull()) { throw new AnalysisException( 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 index 9323b5d2558a53..420bee53e18293 100644 --- 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 @@ -23,7 +23,7 @@ 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 org.apache.doris.thrift.TNullableStringLiteral; import com.google.common.collect.Maps; import org.apache.logging.log4j.LogManager; @@ -97,8 +97,12 @@ public static DateLiteral getRangeEnd(DateLiteral beginTime, FunctionIntervalInf switch (timeUnit) { case "year": return beginTime.plusYears(interval); + case "quarter": + return beginTime.plusMonths(interval * 3); case "month": return beginTime.plusMonths(interval); + case "week": + return beginTime.plusDays(interval * 7); case "day": return beginTime.plusDays(interval); case "hour": @@ -114,7 +118,7 @@ public static DateLiteral getRangeEnd(DateLiteral beginTime, FunctionIntervalInf } public static Map getAddPartitionClauseFromPartitionValues(OlapTable olapTable, - ArrayList> partitionValues, PartitionInfo partitionInfo) + ArrayList> partitionValues, PartitionInfo partitionInfo) throws AnalysisException { Map result = Maps.newHashMap(); ArrayList partitionExprs = partitionInfo.getPartitionExprs(); @@ -124,15 +128,23 @@ public static Map getAddPartitionClauseFromPartition FunctionIntervalInfo intervalInfo = getFunctionIntervalInfo(partitionExprs, partitionType); Set filterPartitionValues = new HashSet(); - for (List partitionValueList : partitionValues) { + for (List partitionValueList : partitionValues) { PartitionKeyDesc partitionKeyDesc = null; String partitionName = "p"; ArrayList curPartitionValues = new ArrayList<>(); - for (TStringLiteral tStringLiteral : partitionValueList) { - curPartitionValues.add(tStringLiteral.value); + for (TNullableStringLiteral tStringLiteral : partitionValueList) { + if (tStringLiteral.is_null) { + if (partitionType == PartitionType.RANGE) { + throw new AnalysisException("Can't create partition for NULL Range"); + } + curPartitionValues.add(null); + } else { + curPartitionValues.add(tStringLiteral.value); + } } + // Concatenate each string with its length. X means null String filterStr = curPartitionValues.stream() - .map(s -> s + s.length()) // Concatenate each string with its length + .map(s -> (s == null) ? "X" : (s + s.length())) .reduce("", (s1, s2) -> s1 + s2); if (filterPartitionValues.contains(filterStr)) { continue; @@ -151,11 +163,14 @@ public static Map getAddPartitionClauseFromPartition List> listValues = new ArrayList<>(); List inValues = new ArrayList<>(); for (String value : curPartitionValues) { - inValues.add(new PartitionValue(value)); + if (value == null) { + inValues.add(new PartitionValue("", true)); + } else { + inValues.add(new PartitionValue(value)); + } } listValues.add(inValues); - partitionKeyDesc = PartitionKeyDesc.createIn( - listValues); + partitionKeyDesc = PartitionKeyDesc.createIn(listValues); partitionName += getFormatPartitionValue(filterStr); if (hasStringType) { if (partitionName.length() > 50) { @@ -179,35 +194,43 @@ public static Map getAddPartitionClauseFromPartition return result; } - public static PartitionKeyDesc createPartitionKeyDescWithRange(DateLiteral beginDateTime, + private 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. + PartitionValue lowerValue = getPartitionFromDate(partitionColumnType, beginDateTime); + PartitionValue upperValue = getPartitionFromDate(partitionColumnType, endDateTime); + return PartitionKeyDesc.createFixed( + Collections.singletonList(lowerValue), + Collections.singletonList(upperValue)); + } + + private static PartitionValue getPartitionFromDate(Type partitionColumnType, DateLiteral dateLiteral) + throws AnalysisException { + // check out of range. + try { + // if lower than range, parse will error. so if hits here, the only possiblility + // is rounding to beyond the limit + dateLiteral.checkValueValid(); + } catch (AnalysisException e) { + return PartitionValue.MAX_VALUE; + } + + String timeString; 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()); + timeString = String.format(DATE_FORMATTER, dateLiteral.getYear(), dateLiteral.getMonth(), + dateLiteral.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()); + timeString = String.format(DATETIME_FORMATTER, + dateLiteral.getYear(), dateLiteral.getMonth(), dateLiteral.getDay(), + dateLiteral.getHour(), dateLiteral.getMinute(), dateLiteral.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)); + + return new PartitionValue(timeString); } - public static String getFormatPartitionValue(String value) { + private static String getFormatPartitionValue(String value) { StringBuilder sb = new StringBuilder(); // When the value is negative if (value.length() > 0 && value.charAt(0) == '-') { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionNames.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionNames.java index ca26a2978e0e54..f82f497176bec2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionNames.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionNames.java @@ -48,7 +48,7 @@ public class PartitionNames implements ParseNode, Writable { // true if these partitions are temp partitions @SerializedName(value = "isTemp") private final boolean isTemp; - private final boolean allPartitions; + private final boolean isStar; private final long count; // Default partition count to collect statistic for external table. private static final long DEFAULT_PARTITION_COUNT = 100; @@ -56,28 +56,28 @@ public class PartitionNames implements ParseNode, Writable { public PartitionNames(boolean isTemp, List partitionNames) { this.partitionNames = partitionNames; this.isTemp = isTemp; - this.allPartitions = false; + this.isStar = false; this.count = 0; } public PartitionNames(PartitionNames other) { this.partitionNames = Lists.newArrayList(other.partitionNames); this.isTemp = other.isTemp; - this.allPartitions = other.allPartitions; + this.isStar = other.isStar; this.count = 0; } - public PartitionNames(boolean allPartitions) { + public PartitionNames(boolean isStar) { this.partitionNames = null; this.isTemp = false; - this.allPartitions = allPartitions; + this.isStar = isStar; this.count = 0; } public PartitionNames(long partitionCount) { this.partitionNames = null; this.isTemp = false; - this.allPartitions = false; + this.isStar = false; this.count = partitionCount; } @@ -89,8 +89,12 @@ public boolean isTemp() { return isTemp; } - public boolean isAllPartitions() { - return allPartitions; + /** + * @return for OLAP table, only in overwrite situation, overwrite auto detect partition + * for External table, all partitions. + */ + public boolean isStar() { + return isStar; } public long getCount() { @@ -99,10 +103,10 @@ public long getCount() { @Override public void analyze(Analyzer analyzer) throws AnalysisException { - if (allPartitions && count > 0) { + if (isStar && count > 0) { throw new AnalysisException("All partition and partition count couldn't be set at the same time."); } - if (allPartitions || count > 0) { + if (isStar || count > 0) { return; } if (partitionNames == null || partitionNames.isEmpty()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionValue.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionValue.java index b20a9035869542..62aa8d18e98801 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionValue.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionValue.java @@ -46,11 +46,13 @@ public PartitionValue(String value, boolean isNullPartition) { } public LiteralExpr getValue(Type type) throws AnalysisException { + if (isNullPartition) { + return new NullLiteral(); + } if (isMax()) { return LiteralExpr.createInfinity(type, true); - } else { - return LiteralExpr.create(value, type); } + return LiteralExpr.create(value, type); } public boolean isMax() { 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 4816c45e9e4f65..723f7dba065c07 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 @@ -1068,6 +1068,14 @@ public Set getPartitionNames() { return Sets.newHashSet(nameToPartition.keySet()); } + public List uncheckedGetPartNamesById(List partitionIds) { + List names = new ArrayList(); + for (Long id : partitionIds) { + names.add(idToPartition.get(id).getName()); + } + return names; + } + public List getPartitionIds() { readLock(); try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java index 0a7c826845097c..b227afdc142eab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java @@ -95,7 +95,8 @@ public static PartitionKey createPartitionKey(List keys, List tasks = Maps.newConcurrentMap(); + // > + // for iot auto detect tasks. a txn will make many task by different rpc + @SerializedName(value = "taskGroups") + private Map> taskGroups = Maps.newConcurrentMap(); + // for one task group, there may be different requests about changing a partition to new. + // but we only change one time and save the relations in partitionPairs. they're protected by taskLocks + @SerializedName(value = "taskLocks") + private Map taskLocks = Maps.newConcurrentMap(); + // > + @SerializedName(value = "partitionPairs") + private Map> partitionPairs = Maps.newConcurrentMap(); + public InsertOverwriteManager() { super("InsertOverwriteDropDirtyPartitions", CLEAN_INTERVAL_SECOND * 1000); } @@ -69,12 +84,85 @@ public long registerTask(long dbId, long tableId, List tempPartitionName return taskId; } + /** + * register insert overwrite task group for auto detect partition. + * it may have many tasks by FrontendService rpc deal. + * all of them will be involved in one txn.(success or fallback) + * + * @return group id, like a transaction id. + */ + public long preRegisterTask() { + long groupId = Env.getCurrentEnv().getNextId(); + taskGroups.put(groupId, new ArrayList()); + taskLocks.put(groupId, new ReentrantLock()); + partitionPairs.put(groupId, Maps.newConcurrentMap()); + return groupId; + } + + /** + * for iot auto detect. register task first. then put in group. + */ + public void registerTaskInGroup(long groupId, long taskId) { + LOG.info("register task " + taskId + " in group " + groupId); + taskGroups.get(groupId).add(taskId); + } + + public List tryReplacePartitionIds(long groupId, List oldPartitionIds) { + Map relations = partitionPairs.get(groupId); + List newIds = new ArrayList(); + for (Long id : oldPartitionIds) { + if (relations.containsKey(id)) { + // if we replaced it. then return new one. + newIds.add(relations.get(id)); + } else { + // otherwise itself. we will deal it soon. + newIds.add(id); + } + } + return newIds; + } + + public void recordPartitionPairs(long groupId, List oldIds, List newIds) { + Map relations = partitionPairs.get(groupId); + Preconditions.checkArgument(oldIds.size() == newIds.size()); + for (int i = 0; i < oldIds.size(); i++) { + relations.put(oldIds.get(i), newIds.get(i)); + } + } + + public ReentrantLock getLock(long groupId) { + return taskLocks.get(groupId); + } + + public void taskGroupFail(long groupId) { + LOG.info("insert overwrite auto detect partition task group [" + groupId + "] failed"); + for (Long taskId : taskGroups.get(groupId)) { + taskFail(taskId); + } + cleanTaskGroup(groupId); + } + + public void taskGroupSuccess(long groupId) { + LOG.info("insert overwrite auto detect partition task group [" + groupId + "] succeed"); + for (Long taskId : taskGroups.get(groupId)) { + taskSuccess(taskId); + } + cleanTaskGroup(groupId); + } + + private void cleanTaskGroup(long groupId) { + partitionPairs.remove(groupId); + taskLocks.remove(groupId); + taskGroups.remove(groupId); + } + /** * when insert overwrite fail, try drop temp partition * * @param taskId */ public void taskFail(long taskId) { + LOG.info("insert overwrite task [" + taskId + "] failed"); boolean rollback = rollback(taskId); if (rollback) { removeTask(taskId); @@ -89,6 +177,7 @@ public void taskFail(long taskId) { * @param taskId */ public void taskSuccess(long taskId) { + LOG.info("insert overwrite task [" + taskId + "] succeed"); removeTask(taskId); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/insertoverwrite/InsertOverwriteUtil.java b/fe/fe-core/src/main/java/org/apache/doris/insertoverwrite/InsertOverwriteUtil.java index af906d2653e7d9..54f9895ab2c977 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/insertoverwrite/InsertOverwriteUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/insertoverwrite/InsertOverwriteUtil.java @@ -83,7 +83,7 @@ public static void replacePartition(OlapTable olapTable, List partitionN } /** - * generate temp partitionName + * generate temp partitionName. must keep same order. * * @param partitionNames * @return diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSink.java index 2a4416686cc9c3..8ecd417691de02 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSink.java @@ -50,6 +50,7 @@ public class UnboundTableSink extends UnboundLogicalSin private final List partitions; private final boolean isPartialUpdate; private final DMLCommandType dmlCommandType; + private final boolean autoDetectPartition; public UnboundTableSink(List nameParts, List colNames, List hints, List partitions, CHILD_TYPE child) { @@ -70,6 +71,25 @@ public UnboundTableSink(List nameParts, List colNames, List nameParts, List colNames, List hints, + boolean temporaryPartition, List partitions, boolean isAutoDetectPartition, + boolean isPartialUpdate, DMLCommandType dmlCommandType, + Optional groupExpression, Optional logicalProperties, + CHILD_TYPE child) { + super(nameParts, PlanType.LOGICAL_UNBOUND_OLAP_TABLE_SINK, ImmutableList.of(), groupExpression, + logicalProperties, colNames, dmlCommandType, child); + this.hints = Utils.copyRequiredList(hints); + this.temporaryPartition = temporaryPartition; + this.partitions = Utils.copyRequiredList(partitions); + this.autoDetectPartition = isAutoDetectPartition; this.isPartialUpdate = isPartialUpdate; this.dmlCommandType = dmlCommandType; } @@ -78,6 +98,10 @@ public boolean isTemporaryPartition() { return temporaryPartition; } + public boolean isAutoDetectPartition() { + return autoDetectPartition; + } + public List getPartitions() { return partitions; } @@ -93,8 +117,8 @@ public boolean isPartialUpdate() { @Override public Plan withChildren(List children) { Preconditions.checkArgument(children.size() == 1, "UnboundOlapTableSink only accepts one child"); - return new UnboundTableSink<>(nameParts, colNames, hints, temporaryPartition, partitions, isPartialUpdate, - dmlCommandType, groupExpression, Optional.empty(), children.get(0)); + return new UnboundTableSink<>(nameParts, colNames, hints, temporaryPartition, partitions, autoDetectPartition, + isPartialUpdate, dmlCommandType, groupExpression, Optional.empty(), children.get(0)); } @Override @@ -134,14 +158,14 @@ public int hashCode() { @Override public Plan withGroupExpression(Optional groupExpression) { - return new UnboundTableSink<>(nameParts, colNames, hints, temporaryPartition, partitions, isPartialUpdate, - dmlCommandType, groupExpression, Optional.of(getLogicalProperties()), child()); + return new UnboundTableSink<>(nameParts, colNames, hints, temporaryPartition, partitions, autoDetectPartition, + isPartialUpdate, dmlCommandType, groupExpression, Optional.of(getLogicalProperties()), child()); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new UnboundTableSink<>(nameParts, colNames, hints, temporaryPartition, partitions, + return new UnboundTableSink<>(nameParts, colNames, hints, temporaryPartition, partitions, autoDetectPartition, isPartialUpdate, dmlCommandType, groupExpression, logicalProperties, children.get(0)); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSinkCreator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSinkCreator.java index 335d2f58035d62..974bec90a2cdcc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSinkCreator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSinkCreator.java @@ -22,6 +22,8 @@ import org.apache.doris.datasource.CatalogIf; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.datasource.hive.HMSExternalCatalog; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.exceptions.ParseException; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.commands.info.DMLCommandType; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; @@ -29,6 +31,8 @@ import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.qe.ConnectContext; +import com.google.common.collect.ImmutableList; + import java.util.List; import java.util.Optional; @@ -71,4 +75,37 @@ public static LogicalSink createUnboundTableSink(List na } throw new RuntimeException("Load data to " + curCatalog.getClass().getSimpleName() + " is not supported."); } + + /** + * create unbound sink for DML plan with auto detect overwrite partition enable. + */ + public static LogicalSink createUnboundTableSinkMaybeOverwrite(List nameParts, + List colNames, List hints, boolean temporaryPartition, List partitions, + boolean isAutoDetectPartition, boolean isOverwrite, boolean isPartialUpdate, DMLCommandType dmlCommandType, + LogicalPlan plan) { + if (isAutoDetectPartition) { // partitions is null + if (!isOverwrite) { + throw new ParseException("ASTERISK is only supported in overwrite partition for OLAP table"); + } + temporaryPartition = false; + partitions = ImmutableList.of(); + } + + String catalogName = RelationUtil.getQualifierName(ConnectContext.get(), nameParts).get(0); + CatalogIf curCatalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogName); + if (curCatalog instanceof InternalCatalog) { + return new UnboundTableSink<>(nameParts, colNames, hints, temporaryPartition, partitions, + isAutoDetectPartition, + isPartialUpdate, dmlCommandType, Optional.empty(), + Optional.empty(), plan); + } else if (curCatalog instanceof HMSExternalCatalog && !isAutoDetectPartition) { + return new UnboundHiveTableSink<>(nameParts, colNames, hints, partitions, + dmlCommandType, Optional.empty(), Optional.empty(), plan); + } + throw new AnalysisException( + "Auto overwrite data to " + curCatalog.getClass().getSimpleName() + " is not supported." + + (isAutoDetectPartition + ? " PARTITION(*) is only supported in overwrite partition for OLAP table" + : "")); + } } 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 9d5af19e0ee9a7..66d4eebecff047 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 @@ -527,14 +527,18 @@ public LogicalPlan visitInsertTable(InsertTableContext ctx) { Optional labelName = ctx.labelName == null ? Optional.empty() : Optional.of(ctx.labelName.getText()); List colNames = ctx.cols == null ? ImmutableList.of() : visitIdentifierList(ctx.cols); // TODO visit partitionSpecCtx - Pair> partitionSpec = visitPartitionSpec(ctx.partitionSpec()); LogicalPlan plan = visitQuery(ctx.query()); - LogicalSink sink = UnboundTableSinkCreator.createUnboundTableSink( + // partitionSpec may be NULL. means auto detect partition. only available when IOT + Pair> partitionSpec = visitPartitionSpec(ctx.partitionSpec()); + boolean isAutoDetect = partitionSpec.second == null; + LogicalSink sink = UnboundTableSinkCreator.createUnboundTableSinkMaybeOverwrite( tableName.build(), colNames, - ImmutableList.of(), - partitionSpec.first, - partitionSpec.second, + ImmutableList.of(), // hints + partitionSpec.first, // isTemp + partitionSpec.second, // partition names + isAutoDetect, + isOverwrite, ConnectContext.get().getSessionVariable().isEnableUniqueKeyPartialUpdate(), DMLCommandType.INSERT, plan); @@ -563,7 +567,9 @@ public Pair> visitPartitionSpec(PartitionSpecContext ctx) boolean temporaryPartition = false; if (ctx != null) { temporaryPartition = ctx.TEMPORARY() != null; - if (ctx.partition != null) { + if (ctx.ASTERISK() != null) { + partitions = null; + } else if (ctx.partition != null) { partitions = ImmutableList.of(ctx.partition.getText()); } else { partitions = visitIdentifierList(ctx.partitions); @@ -845,6 +851,10 @@ public LogicalPlan visitUpdate(UpdateContext ctx) { public LogicalPlan visitDelete(DeleteContext ctx) { List tableName = visitMultipartIdentifier(ctx.tableName); Pair> partitionSpec = visitPartitionSpec(ctx.partitionSpec()); + // TODO: now dont support delete auto detect partition. + if (partitionSpec == null) { + throw new ParseException("Now don't support auto detect partitions in deleting", ctx); + } LogicalPlan query = withTableAlias(LogicalPlanBuilderAssistant.withCheckPolicy( new UnboundRelation(StatementScopeIdGenerator.newRelationId(), tableName, partitionSpec.second, partitionSpec.first)), ctx.tableAlias()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/PartitionTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/PartitionTableInfo.java index c16d5a57024a52..3256effdc3f5bb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/PartitionTableInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/PartitionTableInfo.java @@ -135,10 +135,6 @@ private void validatePartitionColumn(ColumnDefinition column, ConnectContext ctx throw new AnalysisException("Complex type column can't be partition column: " + column.getType().toString()); } - // prohibit to create auto partition with null column anyhow - if (this.isAutoPartition && column.isNullable()) { - throw new AnalysisException("The auto partition column must be NOT NULL"); - } if (!ctx.getSessionVariable().isAllowPartitionColumnNullable() && column.isNullable()) { throw new AnalysisException( "The partition column must be NOT NULL with allow_partition_column_nullable OFF"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java index 2c0f8c13939aa1..4b7b940c9f68f8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java @@ -464,7 +464,7 @@ private EvaluateRangeResult evaluateChildrenThenThis(Expression expr, EvaluateRa for (int i = 0; i < children.size(); i++) { Expression child = children.get(i); EvaluateRangeResult childResult = child.accept(this, context); - if (childResult.result != child) { + if (!childResult.result.equals(child)) { hasNewChildren = true; } childrenResults.add(childResult); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java index 04cef999fda27c..4a825d7956b839 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java @@ -104,7 +104,7 @@ public Expression visitComparisonPredicate(ComparisonPredicate cp, Void context) public List prune() { Builder scanPartitionIds = ImmutableList.builder(); for (OnePartitionEvaluator partition : partitions) { - if (!canPrune(partition)) { + if (!canBePrunedOut(partition)) { scanPartitionIds.add(partition.getPartitionId()); } } @@ -154,14 +154,19 @@ public static final OnePartitionEvaluator toPartitionEvaluator(long id, Partitio } } - private boolean canPrune(OnePartitionEvaluator evaluator) { + /** + * return true if partition is not qualified. that is, can be pruned out. + */ + private boolean canBePrunedOut(OnePartitionEvaluator evaluator) { List> onePartitionInputs = evaluator.getOnePartitionInputs(); for (Map currentInputs : onePartitionInputs) { + // evaluate wether there's possible for this partition to accept this predicate Expression result = evaluator.evaluateWithDefaultPartition(partitionPredicate, currentInputs); if (!result.equals(BooleanLiteral.FALSE) && !(result instanceof NullLiteral)) { return false; } } + // only have false result: Can be pruned out. have other exprs: CanNot be pruned out return true; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java index 061666c681f435..53cfe1e1835bb8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.trees.expressions.literal; import org.apache.doris.analysis.LiteralExpr; +import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.exceptions.UnboundException; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -75,6 +76,10 @@ private void roundMicroSecond(int scale) { this.second = localDateTime.getSecond(); this.microSecond -= 1000000; } + if (checkRange() || checkDate()) { + // may fallback to legacy planner. make sure the behaviour of rounding is same. + throw new AnalysisException("datetime literal [" + toString() + "] is out of range"); + } } public String getFullMicroSecondValue() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java index 5c727ec2220a51..56b99d8288d9ae 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java @@ -160,6 +160,7 @@ public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor executor throw new AnalysisException("group commit is not supported in Nereids now"); } OlapTable olapTable = (OlapTable) targetTableIf; + // the insertCtx contains some variables to adjust SinkNode insertExecutor = new OlapInsertExecutor(ctx, olapTable, label, planner, insertCtx); boolean isEnableMemtableOnSinkNode = olapTable.getTableProperty().getUseSchemaLightChange() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertOverwriteTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertOverwriteTableCommand.java index 788871c744e384..0741982c968fc9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertOverwriteTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertOverwriteTableCommand.java @@ -87,6 +87,10 @@ public void setLabelName(Optional labelName) { this.labelName = labelName; } + public boolean isAutoDetectOverwrite() { + return ((UnboundTableSink) this.logicalQuery).isAutoDetectPartition(); + } + @Override public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { if (!ctx.getSessionVariable().isEnableNereidsDML()) { @@ -134,17 +138,32 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { if (CollectionUtils.isEmpty(partitionNames)) { partitionNames = Lists.newArrayList(targetTable.getPartitionNames()); } - List tempPartitionNames = InsertOverwriteUtil.generateTempPartitionNames(partitionNames); - long taskId = Env.getCurrentEnv().getInsertOverwriteManager() - .registerTask(targetTable.getDatabase().getId(), targetTable.getId(), tempPartitionNames); + + long taskId = 0; try { - InsertOverwriteUtil.addTempPartitions(targetTable, partitionNames, tempPartitionNames); - insertInto(ctx, executor, tempPartitionNames); - InsertOverwriteUtil.replacePartition(targetTable, partitionNames, tempPartitionNames); - Env.getCurrentEnv().getInsertOverwriteManager().taskSuccess(taskId); + if (isAutoDetectOverwrite()) { + // taskId here is a group id. it contains all replace tasks made and registered in rpc process. + taskId = Env.getCurrentEnv().getInsertOverwriteManager().preRegisterTask(); + // When inserting, BE will call to replace partition by FrontendService. FE do the real + // add&replacement and return replace result. So there's no need to do anything else. + insertInto(ctx, executor, taskId); + Env.getCurrentEnv().getInsertOverwriteManager().taskGroupSuccess(taskId); + } else { + List tempPartitionNames = InsertOverwriteUtil.generateTempPartitionNames(partitionNames); + taskId = Env.getCurrentEnv().getInsertOverwriteManager() + .registerTask(targetTable.getDatabase().getId(), targetTable.getId(), tempPartitionNames); + InsertOverwriteUtil.addTempPartitions(targetTable, partitionNames, tempPartitionNames); + insertInto(ctx, executor, tempPartitionNames); + InsertOverwriteUtil.replacePartition(targetTable, partitionNames, tempPartitionNames); + Env.getCurrentEnv().getInsertOverwriteManager().taskSuccess(taskId); + } } catch (Exception e) { LOG.warn("insert into overwrite failed"); - Env.getCurrentEnv().getInsertOverwriteManager().taskFail(taskId); + if (isAutoDetectOverwrite()) { + Env.getCurrentEnv().getInsertOverwriteManager().taskGroupFail(taskId); + } else { + Env.getCurrentEnv().getInsertOverwriteManager().taskFail(taskId); + } throw e; } finally { ConnectContext.get().setSkipAuth(false); @@ -152,14 +171,15 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { } /** - * insert into select + * insert into select. for sepecified temp partitions * - * @param ctx ctx - * @param executor executor + * @param ctx ctx + * @param executor executor * @param tempPartitionNames tempPartitionNames */ private void insertInto(ConnectContext ctx, StmtExecutor executor, List tempPartitionNames) throws Exception { + // copy sink tot replace by tempPartitions UnboundLogicalSink copySink; InsertCommandContext insertCtx; if (logicalQuery instanceof UnboundTableSink) { @@ -173,9 +193,9 @@ private void insertInto(ConnectContext ctx, StmtExecutor executor, List sink.isPartialUpdate(), sink.getDMLCommandType(), (LogicalPlan) (sink.child(0))); - // for overwrite situation, we disable auto create partition. - insertCtx = new OlapInsertCommandContext(); - ((OlapInsertCommandContext) insertCtx).setAllowAutoPartition(false); + // 1. for overwrite situation, we disable auto create partition. + // 2. we save and pass overwrite auto detect by insertCtx + insertCtx = new OlapInsertCommandContext(false); } else if (logicalQuery instanceof UnboundHiveTableSink) { UnboundHiveTableSink sink = (UnboundHiveTableSink) logicalQuery; copySink = (UnboundLogicalSink) UnboundTableSinkCreator.createUnboundTableSink( @@ -202,6 +222,26 @@ private void insertInto(ConnectContext ctx, StmtExecutor executor, List } } + /** + * insert into auto detect partition. + * + * @param ctx ctx + * @param executor executor + */ + private void insertInto(ConnectContext ctx, StmtExecutor executor, long groupId) throws Exception { + UnboundTableSink sink = (UnboundTableSink) logicalQuery; + // 1. for overwrite situation, we disable auto create partition. + // 2. we save and pass overwrite auto detect by insertCtx + OlapInsertCommandContext insertCtx = new OlapInsertCommandContext(false, sink.isAutoDetectPartition(), groupId); + InsertIntoTableCommand insertCommand = new InsertIntoTableCommand(sink, labelName, Optional.of(insertCtx)); + insertCommand.run(ctx, executor); + if (ctx.getState().getStateType() == MysqlStateType.ERR) { + String errMsg = Strings.emptyToNull(ctx.getState().getErrorMessage()); + LOG.warn("InsertInto state error:{}", errMsg); + throw new UserException(errMsg); + } + } + @Override public Plan getExplainPlan(ConnectContext ctx) { return InsertUtils.getPlanForExplain(ctx, this.logicalQuery); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertCommandContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertCommandContext.java index 23dd8d13d9182f..bebade142d97e1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertCommandContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertCommandContext.java @@ -21,12 +21,32 @@ * For Olap Table */ public class OlapInsertCommandContext extends InsertCommandContext { - private boolean allowAutoPartition = true; + private boolean allowAutoPartition; + private boolean autoDetectOverwrite = false; + private long overwriteGroupId = 0; + + public OlapInsertCommandContext(boolean allowAutoPartition) { + this.allowAutoPartition = allowAutoPartition; + } + + public OlapInsertCommandContext(boolean allowAutoPartition, boolean autoDetectOverwrite, long overwriteGroupId) { + this.allowAutoPartition = allowAutoPartition; + this.autoDetectOverwrite = autoDetectOverwrite; + this.overwriteGroupId = overwriteGroupId; + } public boolean isAllowAutoPartition() { return allowAutoPartition; } + public boolean isAutoDetectOverwrite() { + return autoDetectOverwrite; + } + + public long getOverwriteGroupId() { + return overwriteGroupId; + } + public void setAllowAutoPartition(boolean allowAutoPartition) { this.allowAutoPartition = allowAutoPartition; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java index b24426b1e879a5..bb61c71b2826a9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java @@ -100,7 +100,7 @@ public void finalizeSink(PlanFragment fragment, DataSink sink, PhysicalSink phys OlapTableSink olapTableSink = (OlapTableSink) sink; PhysicalOlapTableSink physicalOlapTableSink = (PhysicalOlapTableSink) physicalSink; OlapInsertCommandContext olapInsertCtx = (OlapInsertCommandContext) insertCtx.orElse( - new OlapInsertCommandContext()); + new OlapInsertCommandContext(true)); boolean isStrictMode = ctx.getSessionVariable().getEnableInsertStrict() && physicalOlapTableSink.isPartialUpdate() @@ -113,10 +113,15 @@ public void finalizeSink(PlanFragment fragment, DataSink sink, PhysicalSink phys ctx.getSessionVariable().getSendBatchParallelism(), false, isStrictMode); + // complete and set commands both modify thrift struct olapTableSink.complete(new Analyzer(Env.getCurrentEnv(), ctx)); if (!olapInsertCtx.isAllowAutoPartition()) { olapTableSink.setAutoPartition(false); } + if (olapInsertCtx.isAutoDetectOverwrite()) { + olapTableSink.setAutoDetectOverwite(true); + olapTableSink.setOverwriteGroupId(olapInsertCtx.getOverwriteGroupId()); + } // update // set schema and partition info for tablet id shuffle exchange 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 8406cb53636d3c..d180c7233985cc 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 @@ -180,6 +180,14 @@ public void setAutoPartition(boolean var) { tDataSink.getOlapTableSink().getPartition().setEnableAutomaticPartition(var); } + public void setAutoDetectOverwite(boolean var) { + tDataSink.getOlapTableSink().getPartition().setEnableAutoDetectOverwrite(var); + } + + public void setOverwriteGroupId(long var) { + tDataSink.getOlapTableSink().getPartition().setOverwriteGroupId(var); + } + // must called after tupleDescriptor is computed public void complete(Analyzer analyzer) throws UserException { for (Long partitionId : partitionIds) { @@ -459,16 +467,19 @@ public TOlapTablePartitionParam createPartition(long dbId, OlapTable table, Anal return partitionParam; } - public static void setPartitionKeys(TOlapTablePartition tPartition, PartitionItem partitionItem, int partColNum) { + public static void setPartitionKeys(TOlapTablePartition tPartition, PartitionItem partitionItem, int partColNum) + throws UserException { if (partitionItem instanceof RangePartitionItem) { Range range = partitionItem.getItems(); - // set start keys + // set start keys. min value is a REAL value. should be legal. if (range.hasLowerBound() && !range.lowerEndpoint().isMinValue()) { for (int i = 0; i < partColNum; i++) { tPartition.addToStartKeys(range.lowerEndpoint().getKeys().get(i).treeToThrift().getNodes().get(0)); } } - // set end keys + // TODO: support real MaxLiteral in thrift. + // now we dont send it to BE. if BE meet it, treat it as default value. + // see VOlapTablePartition's ctor in tablet_info.h if (range.hasUpperBound() && !range.upperEndpoint().isMaxValue()) { for (int i = 0; i < partColNum; i++) { tPartition.addToEndKeys(range.upperEndpoint().getKeys().get(i).treeToThrift().getNodes().get(0)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index 2c16b48601b79d..8e33c9cde85373 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -2599,7 +2599,10 @@ private void handleIotStmt() { ConnectContext.get().setSkipAuth(true); try { InsertOverwriteTableStmt iotStmt = (InsertOverwriteTableStmt) this.parsedStmt; - if (iotStmt.getPartitionNames().size() == 0) { + if (iotStmt.isAutoDetectPartition()) { + // insert overwrite table auto detect which partitions need to replace + handleAutoOverwritePartition(iotStmt); + } else if (iotStmt.getPartitionNames().size() == 0) { // insert overwrite table handleOverwriteTable(iotStmt); } else if (iotStmt.isAutoDetectPartition()) { @@ -2751,6 +2754,26 @@ private void handleOverwritePartition(InsertOverwriteTableStmt iotStmt) { } } + /* + * TODO: support insert overwrite auto detect partition in legacy planner + */ + private void handleAutoOverwritePartition(InsertOverwriteTableStmt iotStmt) { + // TODO: + TableName targetTableName = new TableName(null, iotStmt.getDb(), iotStmt.getTbl()); + try { + parsedStmt = new NativeInsertStmt(targetTableName, null, new LabelName(iotStmt.getDb(), iotStmt.getLabel()), + iotStmt.getQueryStmt(), iotStmt.getHints(), iotStmt.getCols(), true).withAutoDetectOverwrite(); + parsedStmt.setUserInfo(context.getCurrentUserIdentity()); + execute(); + } catch (Exception e) { + LOG.warn("IOT insert data error, stmt={}", parsedStmt.toSql(), e); + context.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, "Unexpected exception: " + e.getMessage()); + handleIotRollback(targetTableName); + return; + } + + } + private void handleIotRollback(TableName table) { // insert error drop the tmp table DropTableStmt dropTableStmt = new DropTableStmt(true, table, 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 f51044ec5dc13f..cbb2dd26462299 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 @@ -68,6 +68,8 @@ import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.ExternalDatabase; import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.insertoverwrite.InsertOverwriteManager; +import org.apache.doris.insertoverwrite.InsertOverwriteUtil; import org.apache.doris.load.routineload.ErrorReason; import org.apache.doris.load.routineload.RoutineLoadJob; import org.apache.doris.load.routineload.RoutineLoadJob.JobState; @@ -184,6 +186,7 @@ import org.apache.doris.thrift.TMySqlLoadAcquireTokenResult; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TNodeInfo; +import org.apache.doris.thrift.TNullableStringLiteral; import org.apache.doris.thrift.TOlapTableIndexTablets; import org.apache.doris.thrift.TOlapTablePartition; import org.apache.doris.thrift.TPipelineFragmentParams; @@ -196,6 +199,8 @@ import org.apache.doris.thrift.TPrivilegeType; import org.apache.doris.thrift.TQueryStatsResult; import org.apache.doris.thrift.TQueryType; +import org.apache.doris.thrift.TReplacePartitionRequest; +import org.apache.doris.thrift.TReplacePartitionResult; import org.apache.doris.thrift.TReplicaInfo; import org.apache.doris.thrift.TReportExecStatusParams; import org.apache.doris.thrift.TReportExecStatusResult; @@ -216,7 +221,6 @@ 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; @@ -261,7 +265,9 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; +import java.util.stream.IntStream; // Frontend service used to serve all request for this frontend through // thrift protocol @@ -3343,7 +3349,7 @@ public TCreatePartitionResult createPartition(TCreatePartitionRequest request) t OlapTable olapTable = (OlapTable) table; PartitionInfo partitionInfo = olapTable.getPartitionInfo(); - ArrayList> partitionValues = new ArrayList<>(); + ArrayList> partitionValues = new ArrayList<>(); for (int i = 0; i < request.partitionValues.size(); i++) { if (partitionInfo.getType() == PartitionType.RANGE && request.partitionValues.get(i).size() != 1) { errorStatus.setErrorMsgs( @@ -3402,8 +3408,14 @@ public TCreatePartitionResult createPartition(TCreatePartitionRequest request) t TOlapTablePartition tPartition = new TOlapTablePartition(); tPartition.setId(partition.getId()); int partColNum = partitionInfo.getPartitionColumns().size(); - // set partition keys - OlapTableSink.setPartitionKeys(tPartition, partitionInfo.getItem(partition.getId()), partColNum); + try { + OlapTableSink.setPartitionKeys(tPartition, partitionInfo.getItem(partition.getId()), partColNum); + } catch (UserException ex) { + errorStatus.setErrorMsgs(Lists.newArrayList(ex.getMessage())); + result.setStatus(errorStatus); + LOG.warn("send create partition error status: {}", result); + return result; + } 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())))); @@ -3445,6 +3457,160 @@ public TCreatePartitionResult createPartition(TCreatePartitionRequest request) t return result; } + @Override + public TReplacePartitionResult replacePartition(TReplacePartitionRequest request) throws TException { + LOG.info("Receive replace partition request: {}", request); + long dbId = request.getDbId(); + long tableId = request.getTableId(); + List partitionIds = request.getPartitionIds(); + long taskGroupId = request.getOverwriteGroupId(); + TReplacePartitionResult result = new TReplacePartitionResult(); + TStatus errorStatus = new TStatus(TStatusCode.RUNTIME_ERROR); + if (!Env.getCurrentEnv().isMaster()) { + errorStatus.setStatusCode(TStatusCode.NOT_MASTER); + errorStatus.addToErrorMsgs(NOT_MASTER_ERR_MSG); + LOG.warn("failed to createPartition: {}", NOT_MASTER_ERR_MSG); + return result; + } + + 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); + LOG.warn("send replace partition error status: {}", result); + 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); + LOG.warn("send replace partition error status: {}", result); + 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); + LOG.warn("send replace partition error status: {}", result); + return result; + } + + OlapTable olapTable = (OlapTable) table; + InsertOverwriteManager overwriteManager = Env.getCurrentEnv().getInsertOverwriteManager(); + ReentrantLock taskLock = overwriteManager.getLock(taskGroupId); + List allReqPartNames; // all request partitions + try { + taskLock.lock(); + // we dont lock the table. other thread in this txn will be controled by taskLock. + // if we have already replaced. dont do it again, but acquire the recorded new partition directly. + // if not by this txn, just let it fail naturally is ok. + List replacedPartIds = overwriteManager.tryReplacePartitionIds(taskGroupId, partitionIds); + // here if replacedPartIds still have null. this will throw exception. + allReqPartNames = olapTable.uncheckedGetPartNamesById(replacedPartIds); + + List pendingPartitionIds = IntStream.range(0, partitionIds.size()) + .filter(i -> partitionIds.get(i) == replacedPartIds.get(i)) // equal means not replaced + .mapToObj(partitionIds::get) + .collect(Collectors.toList()); + // from here we ONLY deal the pending partitions. not include the dealed(by others). + if (!pendingPartitionIds.isEmpty()) { + // below two must have same order inner. + List pendingPartitionNames = olapTable.uncheckedGetPartNamesById(pendingPartitionIds); + List tempPartitionNames = InsertOverwriteUtil + .generateTempPartitionNames(pendingPartitionNames); + + long taskId = overwriteManager.registerTask(dbId, tableId, tempPartitionNames); + overwriteManager.registerTaskInGroup(taskGroupId, taskId); + InsertOverwriteUtil.addTempPartitions(olapTable, pendingPartitionNames, tempPartitionNames); + InsertOverwriteUtil.replacePartition(olapTable, pendingPartitionNames, tempPartitionNames); + // now temp partitions are bumped up and use new names. we get their ids and record them. + List newPartitionIds = new ArrayList(); + for (String newPartName : pendingPartitionNames) { + newPartitionIds.add(olapTable.getPartition(newPartName).getId()); + } + overwriteManager.recordPartitionPairs(taskGroupId, pendingPartitionIds, newPartitionIds); + if (LOG.isDebugEnabled()) { + LOG.debug("partition replacement: "); + for (int i = 0; i < pendingPartitionIds.size(); i++) { + LOG.debug("[" + pendingPartitionIds.get(i) + ", " + newPartitionIds.get(i) + "], "); + } + } + } + } catch (DdlException ex) { + errorStatus.setErrorMsgs(Lists.newArrayList(ex.getMessage())); + result.setStatus(errorStatus); + LOG.warn("send create partition error status: {}", result); + return result; + } finally { + taskLock.unlock(); + } + + // build partition & tablets. now all partitions in allReqPartNames are replaced an recorded. + // so they won't be changed again. if other transaction changing it. just let it fail. + List partitions = Lists.newArrayList(); + List tablets = Lists.newArrayList(); + PartitionInfo partitionInfo = olapTable.getPartitionInfo(); + for (String partitionName : allReqPartNames) { + Partition partition = table.getPartition(partitionName); + TOlapTablePartition tPartition = new TOlapTablePartition(); + tPartition.setId(partition.getId()); + + // set partition keys + int partColNum = partitionInfo.getPartitionColumns().size(); + try { + OlapTableSink.setPartitionKeys(tPartition, partitionInfo.getItem(partition.getId()), partColNum); + } catch (UserException ex) { + errorStatus.setErrorMsgs(Lists.newArrayList(ex.getMessage())); + result.setStatus(errorStatus); + LOG.warn("send replace partition error status: {}", result); + return result; + } + 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); + + // successfully return + result.setStatus(new TStatus(TStatusCode.OK)); + if (LOG.isDebugEnabled()) { + LOG.debug("send replace partition result: {}", result); + } + return result; + } + public TGetMetaResult getMeta(TGetMetaRequest request) throws TException { String clientAddr = getClientAddrAsString(); if (LOG.isDebugEnabled()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java index ebd16b3ebe8de9..11bc0510a388ee 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisManager.java @@ -300,7 +300,7 @@ public AnalysisInfo buildAnalysisJobInfo(AnalyzeTblStmt stmt) throws DdlExceptio Set partitionNames = stmt.getPartitionNames(); boolean partitionOnly = stmt.isPartitionOnly(); boolean isSamplingPartition = stmt.isSamplingPartition(); - boolean isAllPartition = stmt.isAllPartitions(); + boolean isAllPartition = stmt.isStarPartition(); long partitionCount = stmt.getPartitionCount(); int samplePercent = stmt.getSamplePercent(); int sampleRows = stmt.getSampleRows(); 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 index 1306a2bff88a3b..ed7cc927273e1a 100644 --- 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 @@ -37,9 +37,9 @@ import org.apache.doris.thrift.TGetDbsResult; import org.apache.doris.thrift.TMetadataTableRequestParams; import org.apache.doris.thrift.TMetadataType; +import org.apache.doris.thrift.TNullableStringLiteral; import org.apache.doris.thrift.TSchemaTableName; import org.apache.doris.thrift.TStatusCode; -import org.apache.doris.thrift.TStringLiteral; import org.apache.doris.utframe.UtFrameUtils; import mockit.Mocked; @@ -109,10 +109,10 @@ public void testCreatePartitionRange() throws Exception { Database db = Env.getCurrentInternalCatalog().getDbOrAnalysisException("test"); OlapTable table = (OlapTable) db.getTableOrAnalysisException("partition_range"); - List> partitionValues = new ArrayList<>(); - List values = new ArrayList<>(); + List> partitionValues = new ArrayList<>(); + List values = new ArrayList<>(); - TStringLiteral start = new TStringLiteral(); + TNullableStringLiteral start = new TNullableStringLiteral(); start.setValue("2023-08-07 00:00:00"); values.add(start); @@ -148,10 +148,10 @@ public void testCreatePartitionList() throws Exception { Database db = Env.getCurrentInternalCatalog().getDbOrAnalysisException("test"); OlapTable table = (OlapTable) db.getTableOrAnalysisException("partition_list"); - List> partitionValues = new ArrayList<>(); - List values = new ArrayList<>(); + List> partitionValues = new ArrayList<>(); + List values = new ArrayList<>(); - TStringLiteral start = new TStringLiteral(); + TNullableStringLiteral start = new TNullableStringLiteral(); start.setValue("BEIJING"); values.add(start); diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift index 909739168c5724..bd7958df8e85dc 100644 --- a/gensrc/thrift/Descriptors.thrift +++ b/gensrc/thrift/Descriptors.thrift @@ -205,6 +205,9 @@ struct TOlapTablePartitionParam { 8: optional list partition_function_exprs 9: optional bool enable_automatic_partition 10: optional Partitions.TPartitionType partition_type + // insert overwrite partition(*) + 11: optional bool enable_auto_detect_overwrite + 12: optional i64 overwrite_group_id } struct TOlapTableIndex { diff --git a/gensrc/thrift/Exprs.thrift b/gensrc/thrift/Exprs.thrift index 9894c1539e82b6..6b7abc2ad939f0 100644 --- a/gensrc/thrift/Exprs.thrift +++ b/gensrc/thrift/Exprs.thrift @@ -191,6 +191,11 @@ struct TStringLiteral { 1: required string value; } +struct TNullableStringLiteral { + 1: optional string value; + 2: optional bool is_null = false; +} + struct TJsonLiteral { 1: required string value; } diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 320149855b4286..150783dcae3488 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -1254,7 +1254,7 @@ struct TCreatePartitionRequest { 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 + 4: optional list> partitionValues } struct TCreatePartitionResult { @@ -1264,6 +1264,21 @@ struct TCreatePartitionResult { 4: optional list nodes } +// these two for auto detect replacing partition +struct TReplacePartitionRequest { + 1: optional i64 overwrite_group_id + 2: optional i64 db_id + 3: optional i64 table_id + 4: optional list partition_ids // partition to replace. +} + +struct TReplacePartitionResult { + 1: optional Status.TStatus status + 2: optional list partitions + 3: optional list tablets + 4: optional list nodes +} + struct TGetMetaReplica { 1: optional i64 id } @@ -1472,6 +1487,8 @@ service FrontendService { TAutoIncrementRangeResult getAutoIncrementRange(1: TAutoIncrementRangeRequest request) TCreatePartitionResult createPartition(1: TCreatePartitionRequest request) + // insert overwrite partition(*) + TReplacePartitionResult replacePartition(1: TReplacePartitionRequest request) TGetMetaResult getMeta(1: TGetMetaRequest request) diff --git a/regression-test/data/insert_overwrite_p0/insert_overwrite_auto_detect.out b/regression-test/data/insert_overwrite_p0/insert_overwrite_auto_detect.out new file mode 100644 index 00000000000000..51e06f3f7bfe85 --- /dev/null +++ b/regression-test/data/insert_overwrite_p0/insert_overwrite_auto_detect.out @@ -0,0 +1,96 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +1 +2 +15 +100 +200 + +-- !sql -- +3 +15 +1234 + +-- !sql -- +1 +2 +15 +333 +444 +555 + +-- !sql -- +-100 +-100 +15 +333 +444 +555 + +-- !sql -- +-100 +-100 +15 +333 +444 +555 + +-- !sql -- +1234567 +Beijing +Shanghai +list +xxx + +-- !sql -- +1234567 +BEIJING +Shanghai +list +xxx + +-- !sql -- +7654321 +7654321 +7654321 +BEIJING +Shanghai +list +xxx + +-- !sql -- +7654321 +BEIJING +LIST +LIST +Shanghai +list +list +xxx + +-- !sql -- +7654321 +BEIJING +LIST +SHANGHAI +XXX + +-- !sql -- +7654321 +BEIJING +LIST +SHANGHAI +XXX + +-- !sql -- +Beijing + +-- !sql -- +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa + +-- !sql -- +2008-01-01 +2008-02-02 +2013-02-02 +2022-03-03 + diff --git a/regression-test/data/insert_overwrite_p1/test_iot_auto_detect_concurrent.out b/regression-test/data/insert_overwrite_p1/test_iot_auto_detect_concurrent.out new file mode 100644 index 00000000000000..3c8a25336daf5d --- /dev/null +++ b/regression-test/data/insert_overwrite_p1/test_iot_auto_detect_concurrent.out @@ -0,0 +1,10 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql3 -- +100 + +-- !sql4 -- +100 + +-- !sql5 -- +0 + 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 index 53eeb4932577b3..dd39f5eafab360 100644 --- 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 @@ -1,5 +1,8 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !sql1 -- +\N +\N + Abc Beijing Beijing @@ -7,6 +10,45 @@ XXX xxx -- !sql2 -- +\N +\N +\N + +Abc +Abc +Beijing +Beijing +Beijing +Beijing +XXX +XXX +new +xxx +xxx + +-- !sql_null0 -- + +Abc +Abc +Beijing +Beijing +Beijing +Beijing +XXX +XXX +new +xxx +xxx + +-- !sql_null1 -- +\N +\N +\N + +-- !sql_null2 -- + + +-- !sql_null3 -- Abc Abc Beijing @@ -43,3 +85,13 @@ x x y +-- !sql_multi_col1 -- +\N \N \N +\N \N \N +-3 \N qwe +-3 \N qwe +-3 -3 qwe +-1 -1 vdf +1 1 asd +2 2 xxx + diff --git a/regression-test/data/partition_p0/auto_partition/test_auto_partition_behavior.out b/regression-test/data/partition_p0/auto_partition/test_auto_partition_behavior.out index 5fe25cce3d8643..40ae8a97a89c1b 100644 --- a/regression-test/data/partition_p0/auto_partition/test_auto_partition_behavior.out +++ b/regression-test/data/partition_p0/auto_partition/test_auto_partition_behavior.out @@ -108,3 +108,12 @@ Yyy -- !sql_overwrite2 -- Xxx +-- !sql_non_order1 -- +1 2020-12-12T00:00 + +-- !sql_non_order2 -- +2 2023-12-12T00:00 + +-- !sql_non_order3 -- +3 2013-12-12T00:00 + 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 index 5cab9d69b1db5f..93c6d86bf9c734 100644 --- 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 @@ -104,3 +104,7 @@ 2122-12-19T22:22:22.222 2122-12-20T22:22:22.222 +-- !right_bound -- +9999-12-31T23:59:59 +9999-12-31T23:59:59.999999 + diff --git a/regression-test/suites/insert_overwrite_p0/insert_overwrite_auto_detect.groovy b/regression-test/suites/insert_overwrite_p0/insert_overwrite_auto_detect.groovy new file mode 100644 index 00000000000000..4cf5f28e15333f --- /dev/null +++ b/regression-test/suites/insert_overwrite_p0/insert_overwrite_auto_detect.groovy @@ -0,0 +1,155 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_iot_auto_detect") { + // only nereids now + sql """set enable_nereids_planner = true""" + sql """set enable_fallback_to_original_planner = false""" + sql """set enable_nereids_dml = true""" + + // range + sql " drop table if exists range1; " + sql """ + create table range1( + k0 int null + ) + partition by range (k0) + ( + PARTITION p10 values less than (10), + PARTITION p100 values less than (100), + PARTITION pMAX values less than (maxvalue) + ) + DISTRIBUTED BY HASH(`k0`) BUCKETS 1 + properties("replication_num" = "1"); + """ + sql " insert into range1 values (1), (2), (15), (100), (200); " + qt_sql " select * from range1 order by k0; " + sql " insert overwrite table range1 partition(*) values (3), (1234); " + qt_sql " select * from range1 order by k0; " + sql " insert overwrite table range1 partition(*) values (1), (2), (333), (444), (555); " + qt_sql " select * from range1 order by k0; " + sql " insert overwrite table range1 partition(*) values (-100), (-100), (333), (444), (555); " + qt_sql " select * from range1 order by k0; " + sql " insert into range1 values (-12345), (12345); " + sql " insert overwrite table range1 partition(*) values (-100), (-100), (333), (444), (555); " + qt_sql " select * from range1 order by k0; " + + // list + sql " drop table if exists list1; " + sql """ + create table list1( + k0 varchar null + ) + partition by list (k0) + ( + PARTITION p1 values in (("Beijing"), ("BEIJING")), + PARTITION p2 values in (("Shanghai"), ("SHANGHAI")), + PARTITION p3 values in (("xxx"), ("XXX")), + PARTITION p4 values in (("list"), ("LIST")), + PARTITION p5 values in (("1234567"), ("7654321")) + ) + DISTRIBUTED BY HASH(`k0`) BUCKETS 1 + properties("replication_num" = "1"); + """ + sql """ insert into list1 values ("Beijing"),("Shanghai"),("xxx"),("list"),("1234567"); """ + qt_sql " select * from list1 order by k0; " + sql """ insert overwrite table list1 partition(*) values ("BEIJING"); """ + qt_sql " select * from list1 order by k0; " + sql """ insert overwrite table list1 partition(*) values ("7654321"), ("7654321"), ("7654321"); """ + qt_sql " select * from list1 order by k0; " + sql """ insert overwrite table list1 partition(*) values ("7654321"), ("list"), ("list"), ("LIST"), ("LIST"); """ + qt_sql " select * from list1 order by k0; " + sql """ insert overwrite table list1 partition(*) values ("BEIJING"), ("SHANGHAI"), ("XXX"), ("LIST"), ("7654321"); """ + qt_sql " select * from list1 order by k0; " + + // with label - transactions + sql """ insert overwrite table list1 partition(*) with label `txn1` values ("BEIJING"), ("7654321"); """ + sql """ insert overwrite table list1 partition(*) with label `txn2` values ("SHANGHAI"), ("LIST"); """ + sql """ insert overwrite table list1 partition(*) with label `txn3` values ("XXX"); """ + + def max_try_milli_secs = 10000 + while(max_try_milli_secs) { + def result = sql " show load where label like 'txn_' " + if(result[0][2] == "FINISHED" && result[1][2] == "FINISHED" && result[2][2] == "FINISHED" ) { + break + } else { + sleep(1000) // wait 1 second every time + max_try_milli_secs -= 1000 + if(max_try_milli_secs <= 0) { + log.info("result: ${result[0][2]}, ${result[1][2]}, ${result[2][2]}") + fail() + } + } + } + + qt_sql " select * from list1 order by k0; " + + // long partition value + sql " drop table if exists list_long; " + sql """ + create table list_long( + k0 varchar null + ) + partition by list (k0) + ( + PARTITION p1 values in (("Beijing"), ("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa")), + PARTITION p2 values in (("nonono")) + ) + DISTRIBUTED BY HASH(`k0`) BUCKETS 1 + properties("replication_num" = "1"); + """ + sql """ insert into list_long values ("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"); """ + sql """ insert overwrite table list_long partition(*) values ("Beijing"); """ + qt_sql " select * from list_long order by k0; " + sql """ insert overwrite table list_long partition(*) values ("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"); """ + qt_sql " select * from list_long order by k0; " + + // miss partitions + try { + sql """ insert overwrite table list1 partition(*) values ("BEIJING"), ("invalid"); """ + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains('Insert has filtered data in strict mode') || + e.getMessage().contains('Cannot found origin partitions in auto detect overwriting')) + } + + sql " drop table if exists dt; " + sql """ + create table dt( + k0 date null + ) + partition by range (k0) + ( + PARTITION p10 values less than ("2010-01-01"), + PARTITION p100 values less than ("2020-01-01"), + PARTITION pMAX values less than ("2030-01-01") + ) + DISTRIBUTED BY HASH(`k0`) BUCKETS 1 + properties("replication_num" = "1"); + """ + sql """ insert into dt values ("2005-01-01"), ("2013-02-02"), ("2022-03-03"); """ + sql """ insert overwrite table dt partition(*) values ("2008-01-01"), ("2008-02-02"); """ + qt_sql " select * from dt order by k0; " + try { + sql """ insert overwrite table dt partition(*) values ("2023-02-02"), ("3000-12-12"); """ + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains('Insert has filtered data in strict mode') || + e.getMessage().contains('Cannot found origin partitions in auto detect overwriting')) + } + +} diff --git a/regression-test/suites/insert_overwrite_p1/ddl/test_iot_auto_detect_concurrent.sql b/regression-test/suites/insert_overwrite_p1/ddl/test_iot_auto_detect_concurrent.sql new file mode 100644 index 00000000000000..02c3b9037b401e --- /dev/null +++ b/regression-test/suites/insert_overwrite_p1/ddl/test_iot_auto_detect_concurrent.sql @@ -0,0 +1,108 @@ +create table test_concurrent_write( + k0 int null +) +partition by range (k0) +( + PARTITION p10 values less than (10), + PARTITION p20 values less than (20), + PARTITION p30 values less than (30), + PARTITION p40 values less than (40), + PARTITION p50 values less than (50), + PARTITION p60 values less than (60), + PARTITION p70 values less than (70), + PARTITION p80 values less than (80), + PARTITION p90 values less than (90), + PARTITION p100 values less than (100), + PARTITION p110 values less than (110), + PARTITION p120 values less than (120), + PARTITION p130 values less than (130), + PARTITION p140 values less than (140), + PARTITION p150 values less than (150), + PARTITION p160 values less than (160), + PARTITION p170 values less than (170), + PARTITION p180 values less than (180), + PARTITION p190 values less than (190), + PARTITION p200 values less than (200), + PARTITION p210 values less than (210), + PARTITION p220 values less than (220), + PARTITION p230 values less than (230), + PARTITION p240 values less than (240), + PARTITION p250 values less than (250), + PARTITION p260 values less than (260), + PARTITION p270 values less than (270), + PARTITION p280 values less than (280), + PARTITION p290 values less than (290), + PARTITION p300 values less than (300), + PARTITION p310 values less than (310), + PARTITION p320 values less than (320), + PARTITION p330 values less than (330), + PARTITION p340 values less than (340), + PARTITION p350 values less than (350), + PARTITION p360 values less than (360), + PARTITION p370 values less than (370), + PARTITION p380 values less than (380), + PARTITION p390 values less than (390), + PARTITION p400 values less than (400), + PARTITION p410 values less than (410), + PARTITION p420 values less than (420), + PARTITION p430 values less than (430), + PARTITION p440 values less than (440), + PARTITION p450 values less than (450), + PARTITION p460 values less than (460), + PARTITION p470 values less than (470), + PARTITION p480 values less than (480), + PARTITION p490 values less than (490), + PARTITION p500 values less than (500), + PARTITION p510 values less than (510), + PARTITION p520 values less than (520), + PARTITION p530 values less than (530), + PARTITION p540 values less than (540), + PARTITION p550 values less than (550), + PARTITION p560 values less than (560), + PARTITION p570 values less than (570), + PARTITION p580 values less than (580), + PARTITION p590 values less than (590), + PARTITION p600 values less than (600), + PARTITION p610 values less than (610), + PARTITION p620 values less than (620), + PARTITION p630 values less than (630), + PARTITION p640 values less than (640), + PARTITION p650 values less than (650), + PARTITION p660 values less than (660), + PARTITION p670 values less than (670), + PARTITION p680 values less than (680), + PARTITION p690 values less than (690), + PARTITION p700 values less than (700), + PARTITION p710 values less than (710), + PARTITION p720 values less than (720), + PARTITION p730 values less than (730), + PARTITION p740 values less than (740), + PARTITION p750 values less than (750), + PARTITION p760 values less than (760), + PARTITION p770 values less than (770), + PARTITION p780 values less than (780), + PARTITION p790 values less than (790), + PARTITION p800 values less than (800), + PARTITION p810 values less than (810), + PARTITION p820 values less than (820), + PARTITION p830 values less than (830), + PARTITION p840 values less than (840), + PARTITION p850 values less than (850), + PARTITION p860 values less than (860), + PARTITION p870 values less than (870), + PARTITION p880 values less than (880), + PARTITION p890 values less than (890), + PARTITION p900 values less than (900), + PARTITION p910 values less than (910), + PARTITION p920 values less than (920), + PARTITION p930 values less than (930), + PARTITION p940 values less than (940), + PARTITION p950 values less than (950), + PARTITION p960 values less than (960), + PARTITION p970 values less than (970), + PARTITION p980 values less than (980), + PARTITION p990 values less than (990), + PARTITION p1000 values less than (1000) +) +DISTRIBUTED BY HASH(`k0`) BUCKETS 1 +properties("replication_num" = "1"); diff --git a/regression-test/suites/insert_overwrite_p1/test_iot_auto_detect_concurrent.groovy b/regression-test/suites/insert_overwrite_p1/test_iot_auto_detect_concurrent.groovy new file mode 100644 index 00000000000000..200dd874df9540 --- /dev/null +++ b/regression-test/suites/insert_overwrite_p1/test_iot_auto_detect_concurrent.groovy @@ -0,0 +1,124 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_iot_auto_detect_concurrent") { + // only nereids now + sql """set enable_nereids_planner = true""" + sql """set enable_fallback_to_original_planner = false""" + sql """set enable_nereids_dml = true""" + + def db_name = "test_iot_auto_detect_concurrent" + def table_name = "test_concurrent_write" + + sql " create database if not exists test_iot_auto_detect_concurrent; " + sql " use test_iot_auto_detect_concurrent; " + sql " drop table if exists test_concurrent_write; " + sql new File("""${context.file.parent}/ddl/test_iot_auto_detect_concurrent.sql""").text + + def success_status = true + def load_data = { range, offset, expect_success -> + try { + sql " use test_iot_auto_detect_concurrent; " + sql """set enable_nereids_planner = true""" + sql """set enable_fallback_to_original_planner = false""" + sql """set enable_nereids_dml = true""" + sql """ insert overwrite table test_concurrent_write partition(*) + select number*10+${offset} from numbers("number" = "${range}"); + """ + } catch (Exception e) { + if (expect_success) { + success_status = false + log.info("fails one") + } + log.info("successfully catch the failed insert") + return + } + if (!expect_success) { + success_status = false + } + } + + def dropping = true + def drop_partition = { + sql " use test_iot_auto_detect_concurrent; " + while (dropping) { + try { + sql """ alter table test_concurrent_write + drop partition p10, drop partition p20, drop partition p30, drop partition p40, drop partition p50, + drop partition p60, drop partition p70, drop partition p80, drop partition p90, drop partition p100; + """ + } catch (Exception e) {} + } + } + + def result + + + /// same data and partitions + success_status = true + sql """ insert into test_concurrent_write select * from numbers("number" = "1000"); """ + def thread1 = Thread.start { load_data(100, 0, false) } + def thread2 = Thread.start { load_data(100, 0, false) } + def thread3 = Thread.start { load_data(100, 0, false) } + def thread4 = Thread.start { load_data(100, 0, false) } + def thread5 = Thread.start { load_data(100, 0, false) } + thread1.join() + thread2.join() + thread3.join() + thread4.join() + thread5.join() + // suppose result: success zero or one + if (success_status) { // success zero + result = sql " select count(k0) from test_concurrent_write; " + assertEquals(result[0][0], 1000) + result = sql " select count(distinct k0) from test_concurrent_write; " + assertEquals(result[0][0], 1000) + } else { // success one + result = sql " select count(k0) from test_concurrent_write; " + assertEquals(result[0][0], 100) + result = sql " select count(distinct k0) from test_concurrent_write; " + assertEquals(result[0][0], 100) + } + + + /// not same data/partitions + success_status = true + sql """ insert overwrite table test_concurrent_write select * from numbers("number" = "1000"); """ + def thread6 = Thread.start { load_data(50, 0, true) } // 0, 10 ... 490 + def thread7 = Thread.start { load_data(50, 500, true) } // 500, 10 ... 990 + thread6.join() + thread7.join() + // suppose result: Success to overwrite with a multiple of ten values + assertTrue(success_status) + qt_sql3 " select count(k0) from test_concurrent_write; " + qt_sql4 " select count(distinct k0) from test_concurrent_write; " + + + /// with drop partition concurrently + success_status = true + sql """ truncate table test_concurrent_write; """ + def thread10 = Thread.start { drop_partition() } + def thread8 = Thread.start { load_data(100, 0, false) } + def thread9 = Thread.start { load_data(100, 0, false) } + thread8.join() + thread9.join() + dropping = false // stop dropping + thread10.join() + // no success insert occur + assertTrue(success_status) // we concerned about this. no + qt_sql5 " select count(k0) from test_concurrent_write; " +} \ No newline at end of file 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 index af70ca35a87786..7868f1ffb9a514 100644 --- 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 @@ -20,7 +20,7 @@ suite("test_auto_list_partition") { sql "drop table if exists list_table1" sql """ CREATE TABLE `list_table1` ( - `str` varchar not null + `str` varchar ) ENGINE=OLAP DUPLICATE KEY(`str`) COMMENT 'OLAP' @@ -32,20 +32,25 @@ suite("test_auto_list_partition") { "replication_allocation" = "tag.location.default: 1" ); """ - sql """ insert into list_table1 values ("Beijing"), ("XXX"), ("xxx"), ("Beijing"), ("Abc") """ + sql """ insert into list_table1 values ("Beijing"), ("XXX"), ("xxx"), ("Beijing"), ("Abc"), (null) """ + sql """ insert into list_table1 values (null), ("") """ // not same partition qt_sql1 """ select * from list_table1 order by `str` """ def result11 = sql "show partitions from list_table1" - assertEquals(result11.size(), 4) - sql """ insert into list_table1 values ("Beijing"), ("XXX"), ("xxx"), ("Beijing"), ("Abc"), ("new") """ + assertEquals(result11.size(), 6) + sql """ insert into list_table1 values ("Beijing"), ("XXX"), ("xxx"), ("Beijing"), ("Abc"), ("new"), (null) """ qt_sql2 """ select * from list_table1 order by `str` """ def result12 = sql "show partitions from list_table1" - assertEquals(result12.size(), 5) + assertEquals(result12.size(), 7) + qt_sql_null0 " select * from list_table1 where str is not null order by str;" // should have empty string + qt_sql_null1 " select * from list_table1 where str is null order by str;" + qt_sql_null2 """ select * from list_table1 where str = "" order by str; """ + qt_sql_null3 """ select * from list_table1 where str != "" order by str; """ // char sql "drop table if exists list_table2" sql """ CREATE TABLE `list_table2` ( - `ch` char not null + `ch` char ) ENGINE=OLAP DUPLICATE KEY(`ch`) COMMENT 'OLAP' @@ -72,7 +77,7 @@ suite("test_auto_list_partition") { sql """ CREATE TABLE `${tblName3}` ( `k1` INT, - `k2` VARCHAR(50) not null, + `k2` VARCHAR(50), `k3` DATETIMEV2(6) ) ENGINE=OLAP DUPLICATE KEY(`k1`) @@ -85,10 +90,10 @@ suite("test_auto_list_partition") { "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') """ + 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'), (4, null, null) """ def result3 = sql "show partitions from ${tblName3}" logger.info("${result3}") - assertEquals(result3.size(), 4) + assertEquals(result3.size(), 5) // int sql "drop table if exists list_table4" @@ -178,7 +183,7 @@ suite("test_auto_list_partition") { sql "drop table if exists test_bigint" sql """ CREATE TABLE test_bigint ( - k bigint not null + k bigint ) AUTO PARTITION BY LIST (`k`) ( @@ -197,7 +202,7 @@ suite("test_auto_list_partition") { sql "drop table if exists test_smallint" sql """ CREATE TABLE test_smallint ( - k smallint not null + k smallint ) AUTO PARTITION BY LIST (`k`) ( @@ -253,9 +258,9 @@ suite("test_auto_list_partition") { sql "drop table if exists test_list_many_column2" sql """ CREATE TABLE test_list_many_column2 ( - id int not null, - k largeint not null, - str varchar not null + id int, + k largeint, + str varchar ) AUTO PARTITION BY LIST (`id`, `k`, `str`) ( @@ -267,9 +272,24 @@ suite("test_auto_list_partition") { """ sql """ insert into test_list_many_column2 values (1,1,"asd"), (-1,-1,"vdf");""" sql """ insert into test_list_many_column2 values (2,2,"xxx"), (-3,-3,"qwe");""" + sql """ insert into test_list_many_column2 values (null,null,null), (-3,null,"qwe");""" + sql """ insert into test_list_many_column2 values (null,null,null), (-3,null,"qwe");""" + qt_sql_multi_col1 "select * from test_list_many_column2 order by id,k,str" result12 = sql "show partitions from test_list_many_column2" logger.info("${result12}") - assertEquals(result12.size(), 4) + assertEquals(result12.size(), 6) + explain { + sql "select * from test_list_many_column2 where id is null" + contains "partitions=1/6 (pXXX)" + } + explain { + sql "select * from test_list_many_column2 where id is null and k is not null" + contains "VEMPTYSET" + } + explain { + sql "select * from test_list_many_column2 where k is not null" + contains "partitions=4/6" + } sql "drop table if exists stream_load_list_test_table_string_key" sql """ diff --git a/regression-test/suites/partition_p0/auto_partition/test_auto_partition_behavior.groovy b/regression-test/suites/partition_p0/auto_partition/test_auto_partition_behavior.groovy index 41a243501fb758..e3300266ada69f 100644 --- a/regression-test/suites/partition_p0/auto_partition/test_auto_partition_behavior.groovy +++ b/regression-test/suites/partition_p0/auto_partition/test_auto_partition_behavior.groovy @@ -186,66 +186,58 @@ suite("test_auto_partition_behavior") { sql """ insert overwrite table rewrite partition(p1) values ("Xxx") """ qt_sql_overwrite2 """ select * from rewrite """ // Xxx - // prohibit NULLABLE auto partition column - // legacy - sql " set experimental_enable_nereids_planner=false " - test { - sql "drop table if exists test_null1" - sql """ - create table test_null1( - k0 datetime(6) null - ) - auto partition by range date_trunc(k0, 'hour') - ( - ) - DISTRIBUTED BY HASH(`k0`) BUCKETS 2 - properties("replication_num" = "1"); - """ - exception "The auto partition column must be NOT NULL" - } - test { - sql "drop table if exists test_null2" - sql """ - create table test_null2( - k0 int null - ) - auto partition by list (k0) - ( - ) - DISTRIBUTED BY HASH(`k0`) BUCKETS 2 - properties("replication_num" = "1"); + sql " drop table if exists non_order; " + sql """ + CREATE TABLE `non_order` ( + `k0` int not null, + `k1` datetime(6) not null + ) + auto partition by range (date_trunc(`k1`, 'year')) + ( + ) + DISTRIBUTED BY HASH(`k0`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); """ - exception "The auto partition column must be NOT NULL" - } - // nereids + // insert + sql """ insert into non_order values (1, '2020-12-12'); """ + sql """ insert into non_order values (2, '2023-12-12'); """ + sql """ insert into non_order values (3, '2013-12-12'); """ + qt_sql_non_order1 """ select * from non_order where k1 = '2020-12-12'; """ + qt_sql_non_order2 """ select * from non_order where k1 = '2023-12-12'; """ + qt_sql_non_order3 """ select * from non_order where k1 = '2013-12-12'; """ + + // range partition can't auto create null partition sql " set experimental_enable_nereids_planner=true " + sql "drop table if exists invalid_null_range" test { - sql "drop table if exists test_null1" sql """ - create table test_null1( + create table invalid_null_range( k0 datetime(6) null ) - auto partition by range date_trunc(k0, 'hour') + auto partition by range (date_trunc(k0, 'hour')) ( ) DISTRIBUTED BY HASH(`k0`) BUCKETS 2 properties("replication_num" = "1"); """ - exception "The auto partition column must be NOT NULL" + exception "AUTO RANGE PARTITION doesn't support NULL column" } + sql " set experimental_enable_nereids_planner=false " + sql "drop table if exists invalid_null_range" test { - sql "drop table if exists test_null2" sql """ - create table test_null2( - k0 int null + create table invalid_null_range( + k0 datetime(6) null ) - auto partition by list (k0) + auto partition by range (date_trunc(k0, 'hour')) ( ) DISTRIBUTED BY HASH(`k0`) BUCKETS 2 properties("replication_num" = "1"); """ - exception "The auto partition column must be NOT NULL" + exception "AUTO RANGE PARTITION doesn't support NULL column" } // PROHIBIT different timeunit of interval when use both auto & dynamic partition 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 index 6f20a6c33e7a06..d728ba9a1457de 100644 --- 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 @@ -16,10 +16,9 @@ // under the License. suite("test_auto_partition_load") { - def tblName1 = "load_table1" - sql "drop table if exists ${tblName1}" + sql "drop table if exists load_table1" sql """ - CREATE TABLE `${tblName1}` ( + CREATE TABLE `load_table1` ( `k1` INT, `k2` DATETIME NOT NULL, `k3` DATETIMEV2(6) @@ -35,26 +34,25 @@ suite("test_auto_partition_load") { ); """ streamLoad { - table "${tblName1}" + table "load_table1" set 'column_separator', ',' file "auto_partition_stream_load1.csv" time 20000 } - sql """ insert into ${tblName1} values (11, '2007-12-12 12:12:12.123', '2001-11-14 12:12:12.123456') """ - sql """ insert into ${tblName1} values (12, '2008-12-12 12:12:12.123', '2001-11-14 12:12:12.123456') """ - sql """ insert into ${tblName1} values (13, '2003-12-12 12:12:12.123', '2001-11-14 12:12:12.123456') """ - sql """ insert into ${tblName1} values (14, '2002-12-12 12:12:12.123', '2001-11-14 12:12:12.123456') """ + sql """ insert into load_table1 values (11, '2007-12-12 12:12:12.123', '2001-11-14 12:12:12.123456') """ + sql """ insert into load_table1 values (12, '2008-12-12 12:12:12.123', '2001-11-14 12:12:12.123456') """ + sql """ insert into load_table1 values (13, '2003-12-12 12:12:12.123', '2001-11-14 12:12:12.123456') """ + sql """ insert into load_table1 values (14, '2002-12-12 12:12:12.123', '2001-11-14 12:12:12.123456') """ - qt_select1 "select * from ${tblName1} order by k1" - result1 = sql "show partitions from ${tblName1}" + qt_select1 "select * from load_table1 order by k1" + result1 = sql "show partitions from load_table1" logger.info("${result1}") assertEquals(result1.size(), 8) - def tblName2 = "load_table2" - sql "drop table if exists ${tblName2}" + sql "drop table if exists load_table2" sql """ - CREATE TABLE `${tblName2}` ( + CREATE TABLE `load_table2` ( `k1` INT, `k2` VARCHAR(50) not null, `k3` DATETIMEV2(6) @@ -70,18 +68,18 @@ suite("test_auto_partition_load") { ); """ streamLoad { - table "${tblName2}" + table "load_table2" set 'column_separator', ',' file "auto_partition_stream_load2.csv" time 20000 } - sql """ insert into ${tblName2} values (11, '11', '2123-11-14 12:12:12.123456') """ - sql """ insert into ${tblName2} values (12, 'Chengdu', '2123-11-14 12:12:12.123456') """ - sql """ insert into ${tblName2} values (13, '11', '2123-11-14 12:12:12.123456') """ - sql """ insert into ${tblName2} values (14, '12', '2123-11-14 12:12:12.123456') """ + sql """ insert into load_table2 values (11, '11', '2123-11-14 12:12:12.123456') """ + sql """ insert into load_table2 values (12, 'Chengdu', '2123-11-14 12:12:12.123456') """ + sql """ insert into load_table2 values (13, '11', '2123-11-14 12:12:12.123456') """ + sql """ insert into load_table2 values (14, '12', '2123-11-14 12:12:12.123456') """ - qt_select2 "select * from ${tblName2} order by k1" - result2 = sql "show partitions from ${tblName2}" + qt_select2 "select * from load_table2 order by k1" + result2 = sql "show partitions from load_table2" logger.info("${result2}") assertEquals(result2.size(), 11) } 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 index cf11ceacba9ccf..e0f8db3ac8f7e7 100644 --- 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 @@ -16,14 +16,12 @@ // under the License. suite("test_auto_range_partition") { - def tblName1 = "range_table1" - sql "drop table if exists ${tblName1}" + sql "drop table if exists range_table1" sql """ - CREATE TABLE `${tblName1}` ( + CREATE TABLE `range_table1` ( `TIME_STAMP` datetimev2 NOT NULL COMMENT '采集日期' - ) ENGINE=OLAP + ) DUPLICATE KEY(`TIME_STAMP`) - COMMENT 'OLAP' auto partition by range (date_trunc(`TIME_STAMP`, 'day')) ( ) @@ -32,12 +30,12 @@ suite("test_auto_range_partition") { "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') """ + sql """ insert into range_table1 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 range_table1 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 """ + qt_select00 """ select * from range_table1 order by TIME_STAMP """ + qt_select01 """ select * from range_table1 WHERE TIME_STAMP = '2022-12-15' order by TIME_STAMP """ + qt_select02 """ select * from range_table1 WHERE TIME_STAMP > '2022-12-15' order by TIME_STAMP """ sql "drop table if exists range_table_date" sql """ @@ -61,10 +59,9 @@ suite("test_auto_range_partition") { qt_date2 """ select * from range_table_date WHERE TIME_STAMP = '2022-12-15' order by TIME_STAMP """ qt_date3 """ select * from range_table_date WHERE TIME_STAMP > '2022-12-15' order by TIME_STAMP """ - def tblName2 = "range_table2" - sql "drop table if exists ${tblName2}" + sql "drop table if exists range_table2" sql """ - CREATE TABLE `${tblName2}` ( + CREATE TABLE `range_table2` ( `TIME_STAMP` datetimev2(3) NOT NULL COMMENT '采集日期' ) ENGINE=OLAP DUPLICATE KEY(`TIME_STAMP`) @@ -77,35 +74,69 @@ suite("test_auto_range_partition") { "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') """ + sql """ insert into range_table2 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 range_table2 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 range_table2 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 """ + qt_select10 """ select * from range_table2 order by TIME_STAMP """ + qt_select11 """ select * from range_table2 WHERE TIME_STAMP = '2022-12-15 22:22:22.222' order by TIME_STAMP """ + qt_select12 """ select * from range_table2 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 "drop table if exists right_bound" sql """ - CREATE TABLE `${tblName3}` ( - `k1` INT, - `k2` DATETIMEV2(3) NOT NULL, - `k3` DATETIMEV2(6) - ) ENGINE=OLAP - DUPLICATE KEY(`k1`) - COMMENT 'OLAP' - AUTO PARTITION BY RANGE date_trunc(`k2`, 'day') + create table right_bound( + k0 datetime(6) not null + ) + auto partition by range (date_trunc(k0, 'second')) + ( + partition pX values less than ("1970-01-01") + ) + DISTRIBUTED BY HASH(`k0`) BUCKETS auto + properties("replication_num" = "1"); + """ + sql " insert into right_bound values ('9999-12-31 23:59:59'); " + sql " insert into right_bound values ('9999-12-31 23:59:59.999999'); " + qt_right_bound " select * from right_bound order by k0; " + result2 = sql "show partitions from right_bound" + logger.info("${result2}") + assertEquals(result2.size(), 2) + + sql "drop table if exists week_range" + sql """ + CREATE TABLE `week_range` ( + `TIME_STAMP` datev2 NOT NULL + ) + DUPLICATE KEY(`TIME_STAMP`) + auto partition by range (date_trunc(`TIME_STAMP`, 'week')) + ( + ) + DISTRIBUTED BY HASH(`TIME_STAMP`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + sql " insert into week_range values (20240408), (20240409); " + result2 = sql "show partitions from week_range" + logger.info("${result2}") + assertEquals(result2.size(), 1) + + sql "drop table if exists quarter_range" + sql """ + CREATE TABLE `quarter_range` ( + `TIME_STAMP` datev2 NOT NULL + ) + DUPLICATE KEY(`TIME_STAMP`) + auto partition by range (date_trunc(`TIME_STAMP`, 'quarter')) ( ) - DISTRIBUTED BY HASH(`k1`) BUCKETS 16 + DISTRIBUTED BY HASH(`TIME_STAMP`) BUCKETS 10 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) + """ + sql " insert into quarter_range values (20240102), (20240330), (20241001), (20241231); " + result2 = sql "show partitions from quarter_range" + logger.info("${result2}") + assertEquals(result2.size(), 2) } diff --git a/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition_with_rename.groovy b/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition_with_rename.groovy index 6c9d245bef392a..5fd190effc51b9 100644 --- a/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition_with_rename.groovy +++ b/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition_with_rename.groovy @@ -16,10 +16,9 @@ // under the License. suite("test_dynamic_partition_with_rename") { - def tbl = "test_dynamic_partition_with_rename" - sql "drop table if exists ${tbl}" + sql "drop table if exists test_dynamic_partition_with_rename" sql """ - CREATE TABLE IF NOT EXISTS ${tbl} + CREATE TABLE IF NOT EXISTS test_dynamic_partition_with_rename ( k1 date NOT NULL, k2 varchar(20) NOT NULL, k3 int sum NOT NULL ) AGGREGATE KEY(k1,k2) PARTITION BY RANGE(k1) ( ) @@ -34,21 +33,21 @@ suite("test_dynamic_partition_with_rename") { "dynamic_partition.create_history_partition"="true", "dynamic_partition.replication_allocation" = "tag.location.default: 1") """ - def result = sql "show partitions from ${tbl}" + def result = sql "show partitions from test_dynamic_partition_with_rename" assertEquals(7, result.size()) // rename distributed column, then try to add too more dynamic partition - sql "alter table ${tbl} rename column k1 renamed_k1" + sql "alter table test_dynamic_partition_with_rename rename column k1 renamed_k1" sql """ ADMIN SET FRONTEND CONFIG ('dynamic_partition_check_interval_seconds' = '1') """ - sql """ alter table ${tbl} set('dynamic_partition.end'='5') """ - result = sql "show partitions from ${tbl}" + sql """ alter table test_dynamic_partition_with_rename set('dynamic_partition.end'='5') """ + result = sql "show partitions from test_dynamic_partition_with_rename" for (def retry = 0; retry < 120; retry++) { // at most wait 120s if (result.size() == 9) { break; } logger.info("wait dynamic partition scheduler, sleep 1s") sleep(1000); // sleep 1s - result = sql "show partitions from ${tbl}" + result = sql "show partitions from test_dynamic_partition_with_rename" } assertEquals(9, result.size()) for (def line = 0; line < result.size(); line++) { @@ -56,5 +55,5 @@ suite("test_dynamic_partition_with_rename") { assertEquals("renamed_k1", result.get(line).get(7)) } - sql "drop table ${tbl}" + sql "drop table test_dynamic_partition_with_rename" } diff --git a/regression-test/suites/partition_p0/multi_partition/test_multi_column_partition.groovy b/regression-test/suites/partition_p0/multi_partition/test_multi_column_partition.groovy index a42735cabf282c..40c1ba9608a03f 100644 --- a/regression-test/suites/partition_p0/multi_partition/test_multi_column_partition.groovy +++ b/regression-test/suites/partition_p0/multi_partition/test_multi_column_partition.groovy @@ -78,6 +78,7 @@ suite("test_multi_partition_key", "p0") { // partition columns are int & datetime + sql " drop table if exists err_1 " sql """ CREATE TABLE err_1 ( k1 TINYINT NOT NULL, diff --git a/regression-test/suites/partition_p0/test_datev2_partition.groovy b/regression-test/suites/partition_p0/test_datev2_partition.groovy index 63852bb4e2e672..54db28a18c7440 100644 --- a/regression-test/suites/partition_p0/test_datev2_partition.groovy +++ b/regression-test/suites/partition_p0/test_datev2_partition.groovy @@ -16,10 +16,9 @@ // under the License. suite("test_datev2_partition") { - def tblName1 = "test_datev2_partition1" - sql "drop table if exists ${tblName1}" + sql "drop table if exists test_datev2_partition1" sql """ - CREATE TABLE `${tblName1}` ( + CREATE TABLE `test_datev2_partition1` ( `TIME_STAMP` datev2 NOT NULL COMMENT '采集日期' ) ENGINE=OLAP DUPLICATE KEY(`TIME_STAMP`) @@ -38,16 +37,15 @@ suite("test_datev2_partition") { "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 test_datev2_partition1 values ('2022-12-14'), ('2022-12-15'), ('2022-12-16'), ('2022-12-17'), ('2022-12-18'), ('2022-12-19'), ('2022-12-20') """ - 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 """ + qt_select """ select * from test_datev2_partition1 order by TIME_STAMP """ + qt_select """ select * from test_datev2_partition1 WHERE TIME_STAMP = '2022-12-15' order by TIME_STAMP """ + qt_select """ select * from test_datev2_partition1 WHERE TIME_STAMP > '2022-12-15' order by TIME_STAMP """ - def tblName2 = "test_datev2_partition2" - sql "drop table if exists ${tblName2}" + sql "drop table if exists test_datev2_partition2" sql """ - CREATE TABLE `${tblName2}` ( + CREATE TABLE `test_datev2_partition2` ( `TIME_STAMP` datetimev2(3) NOT NULL COMMENT '采集日期' ) ENGINE=OLAP DUPLICATE KEY(`TIME_STAMP`) @@ -66,9 +64,9 @@ suite("test_datev2_partition") { "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 test_datev2_partition2 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') """ - 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 """ + qt_select """ select * from test_datev2_partition2 order by TIME_STAMP """ + qt_select """ select * from test_datev2_partition2 WHERE TIME_STAMP = '2022-12-15 22:22:22.222' order by TIME_STAMP """ + qt_select """ select * from test_datev2_partition2 WHERE TIME_STAMP > '2022-12-15 22:22:22.222' order by TIME_STAMP """ }