From 0b423755000ade4e61161a74c5d2ef32cbd1a9e3 Mon Sep 17 00:00:00 2001 From: Lightman <31928846+Lchangliang@users.noreply.github.com> Date: Wed, 25 May 2022 16:26:50 +0800 Subject: [PATCH] [Schema Change] support fast add/drop column (#49) * [feature](schema-change) support fast schema change. coauthor: yixiutt * [schema change] Using columns desc from fe to read data. coauthor: Lchangliang * [feature](schema change) schema change optimize for add/drop columns. 1.add uniqueId field for class column. 2.schema change for add/drop columns directly update schema meta Co-authored-by: yixiutt Co-authored-by: SWJTU-ZhangLei <1091517373@qq.com> [Feature](schema change) fix write and add regression test (#69) Co-authored-by: yixiutt [schema change] be ssupport that delete use newest schema add delete regression test fix regression case (#107) tmp [feature](schema change) light schema change exclude rollup and agg/uniq/dup key type. [feature](schema change) fe olapTable maxUniqueId write in disk. [feature](schema change) add rpc iface for sc add column. [feature](schema change) add columnsDesc to TPushReq for ligtht sc. resolve the deadlock when schema change (#124) fix columns from fe don't has bitmap_index flag (#134) add update/delete case construct MATERIALIZED schema from origin schema when insert fix not vectorized compaction coredump use segment cache choose newest schema by schema version when compaction (#182) [bugfix](schema change) fix ligth schema change problem. [feature](schema change) light schema change add alter job. (#1) fix be ut [bug] (schema change) unique drop key column should not light schema change [feature](schema change) add schema change regression-test. fix regression test [bugfix](schema change) fix multi alter clauses for light schema change. (#2) [bugfix](schema change) fix multi clauses calculate column unique id (#3) modify PushTask process (#217) [Bugfix](schema change) fix jobId replay cause bdbje exception. [bug](schema change) fix max col unique id repeatitive. (#232) [optimize](schema change) modify pendingMaxColUniqueId generate rule. fix compaction error * fix be ut * fix snapshot load core fix unique_id error (#278) [refact](fe) remove redundant code for light schema change. (#4) [refact](fe) remove redundant code for light schema change. (#4) format fe core format be core fix be ut modify fe meta version fix rebase error flush schema into rowset_meta in old table [refactor](schema change) refact fe light schema change. (#5) delete the change of schemahash and support get max version schema --- be/src/exec/olap_scanner.cpp | 28 +- be/src/exec/olap_scanner.h | 2 + be/src/exec/tablet_info.cpp | 13 + be/src/exec/tablet_info.h | 2 + be/src/olap/base_tablet.cpp | 11 + be/src/olap/base_tablet.h | 4 +- be/src/olap/collect_iterator.cpp | 8 +- be/src/olap/compaction.cpp | 26 +- be/src/olap/compaction.h | 2 +- be/src/olap/convert_rowset.cpp | 141 + be/src/olap/data_dir.cpp | 14 + be/src/olap/delta_writer.cpp | 29 +- be/src/olap/delta_writer.h | 12 +- be/src/olap/merger.cpp | 11 +- be/src/olap/merger.h | 2 + be/src/olap/push_handler.cpp | 76 +- be/src/olap/push_handler.h | 14 +- be/src/olap/reader.cpp | 56 +- be/src/olap/reader.h | 4 + be/src/olap/rowset/beta_rowset.cpp | 5 +- be/src/olap/rowset/beta_rowset.h | 3 +- be/src/olap/rowset/beta_rowset_reader.cpp | 2 +- be/src/olap/rowset/beta_rowset_writer.cpp | 2 + be/src/olap/rowset/rowset.cpp | 7 +- be/src/olap/rowset/rowset.h | 1 + be/src/olap/rowset/rowset_meta.h | 14 + be/src/olap/rowset/segment_v2/segment.cpp | 15 +- be/src/olap/rowset/segment_v2/segment.h | 4 +- .../rowset/segment_v2/segment_iterator.cpp | 2 +- be/src/olap/schema_change.cpp | 86 +- be/src/olap/schema_change.h | 6 +- be/src/olap/segment_loader.cpp | 10 +- be/src/olap/segment_loader.h | 13 +- be/src/olap/tablet.cpp | 3056 +++++++++-------- be/src/olap/tablet.h | 10 +- be/src/olap/tablet_meta.cpp | 18 +- be/src/olap/tablet_meta.h | 8 +- be/src/olap/tablet_schema.cpp | 140 +- be/src/olap/tablet_schema.h | 22 +- be/src/runtime/descriptors.cpp | 2 + be/src/runtime/descriptors.h | 4 + be/src/runtime/tablets_channel.cpp | 2 + be/src/runtime/tablets_channel.h | 1 + be/src/vec/exec/volap_scanner.cpp | 29 +- be/src/vec/exec/volap_scanner.h | 2 + be/src/vec/olap/block_reader.cpp | 2 +- be/src/vec/olap/vcollect_iterator.cpp | 2 +- be/src/vec/olap/vcollect_iterator.h | 2 +- .../olap/test_data/header_without_inc_rs.txt | 12 +- .../apache/doris/common/FeMetaVersion.java | 4 +- .../org/apache/doris/alter/AlterJobV2.java | 8 + .../org/apache/doris/alter/RollupJobV2.java | 2 +- .../doris/alter/SchemaChangeHandler.java | 576 +++- .../apache/doris/alter/SchemaChangeJobV2.java | 14 +- .../org/apache/doris/analysis/ColumnDef.java | 2 +- .../apache/doris/analysis/SlotDescriptor.java | 36 +- .../org/apache/doris/analysis/SlotRef.java | 2 +- .../org/apache/doris/catalog/Catalog.java | 235 +- .../java/org/apache/doris/catalog/Column.java | 34 +- .../doris/catalog/MaterializedIndexMeta.java | 12 + .../org/apache/doris/catalog/OlapTable.java | 22 + .../java/org/apache/doris/catalog/Table.java | 2 +- .../doris/datasource/InternalDataSource.java | 121 +- .../apache/doris/journal/JournalEntity.java | 6 + .../org/apache/doris/load/DeleteHandler.java | 9 +- .../org/apache/doris/load/LoadChecker.java | 8 +- .../doris/load/loadv2/SparkLoadJob.java | 120 +- .../org/apache/doris/persist/EditLog.java | 52 +- .../apache/doris/persist/OperationType.java | 3 + .../persist/TableAddOrDropColumnsInfo.java | 115 + .../apache/doris/planner/OlapScanNode.java | 138 +- .../apache/doris/planner/OlapTableSink.java | 33 +- .../doris/service/FrontendServiceImpl.java | 59 +- .../apache/doris/task/AlterReplicaTask.java | 15 +- .../java/org/apache/doris/task/PushTask.java | 32 +- .../doris/alter/SchemaChangeHandlerTest.java | 326 +- .../TableAddOrDropColumnsInfoTest.java | 97 + gensrc/proto/descriptors.proto | 2 + gensrc/proto/olap_file.proto | 4 + gensrc/thrift/AgentService.thrift | 17 +- gensrc/thrift/Descriptors.thrift | 17 + gensrc/thrift/PlanNodes.thrift | 1 + .../test_compaction_schema_change.out | 11 + .../test_delete_schema_change.out | 44 + .../test_update_schema_change.out | 64 + .../test_agg_keys_schema_change.groovy | 230 ++ .../test_agg_mv_schema_change.groovy | 214 ++ .../test_agg_rollup_schema_change.groovy | 211 ++ .../test_agg_vals_schema_change.groovy | 227 ++ .../test_delete_schema_change.sql | 44 + .../test_dup_keys_schema_change.groovy | 229 ++ .../test_dup_mv_schema_change.groovy | 244 ++ .../test_dup_rollup_schema_change.groovy | 244 ++ .../test_dup_vals_schema_change.groovy | 216 ++ .../test_uniq_keys_schema_change.groovy | 227 ++ .../test_uniq_mv_schema_change.groovy | 231 ++ .../test_uniq_rollup_schema_change.groovy | 243 ++ .../test_uniq_vals_schema_change.groovy | 218 ++ .../test_update_schema_change.sql | 50 + 99 files changed, 6751 insertions(+), 2257 deletions(-) create mode 100644 be/src/olap/convert_rowset.cpp create mode 100644 fe/fe-core/src/main/java/org/apache/doris/persist/TableAddOrDropColumnsInfo.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/persist/TableAddOrDropColumnsInfoTest.java create mode 100644 regression-test/data/schema_change/test_compaction_schema_change.out create mode 100644 regression-test/data/schema_change/test_delete_schema_change.out create mode 100644 regression-test/data/schema_change/test_update_schema_change.out create mode 100644 regression-test/suites/schema_change/test_agg_keys_schema_change.groovy create mode 100644 regression-test/suites/schema_change/test_agg_mv_schema_change.groovy create mode 100644 regression-test/suites/schema_change/test_agg_rollup_schema_change.groovy create mode 100644 regression-test/suites/schema_change/test_agg_vals_schema_change.groovy create mode 100644 regression-test/suites/schema_change/test_delete_schema_change.sql create mode 100644 regression-test/suites/schema_change/test_dup_keys_schema_change.groovy create mode 100644 regression-test/suites/schema_change/test_dup_mv_schema_change.groovy create mode 100644 regression-test/suites/schema_change/test_dup_rollup_schema_change.groovy create mode 100644 regression-test/suites/schema_change/test_dup_vals_schema_change.groovy create mode 100644 regression-test/suites/schema_change/test_uniq_keys_schema_change.groovy create mode 100644 regression-test/suites/schema_change/test_uniq_mv_schema_change.groovy create mode 100644 regression-test/suites/schema_change/test_uniq_rollup_schema_change.groovy create mode 100644 regression-test/suites/schema_change/test_uniq_vals_schema_change.groovy create mode 100644 regression-test/suites/schema_change/test_update_schema_change.sql diff --git a/be/src/exec/olap_scanner.cpp b/be/src/exec/olap_scanner.cpp index e0e6a685497547a..59f6a8ab76dcb85 100644 --- a/be/src/exec/olap_scanner.cpp +++ b/be/src/exec/olap_scanner.cpp @@ -23,7 +23,9 @@ #include "exprs/expr_context.h" #include "gen_cpp/PaloInternalService_types.h" #include "olap/decimal12.h" +#include "olap/field.h" #include "olap/storage_engine.h" +#include "olap/tablet_schema.h" #include "olap/uint24.h" #include "olap_scan_node.h" #include "olap_utils.h" @@ -86,6 +88,14 @@ Status OlapScanner::prepare( LOG(WARNING) << ss.str(); return Status::InternalError(ss.str()); } + _tablet_schema = _tablet->tablet_schema(); + if (!_parent->_olap_scan_node.columns_desc.empty() && + _parent->_olap_scan_node.columns_desc[0].col_unique_id >= 0) { + _tablet_schema.clear_columns(); + for (const auto& column_desc : _parent->_olap_scan_node.columns_desc) { + _tablet_schema.append_column(TabletColumn(column_desc)); + } + } { std::shared_lock rdlock(_tablet->get_header_lock()); const RowsetSharedPtr rowset = _tablet->rowset_with_max_version(); @@ -168,6 +178,7 @@ Status OlapScanner::_init_tablet_reader_params( RETURN_IF_ERROR(_init_return_columns(!_tablet_reader_params.direct_mode)); _tablet_reader_params.tablet = _tablet; + _tablet_reader_params.tablet_schema = &_tablet_schema; _tablet_reader_params.reader_type = READER_QUERY; _tablet_reader_params.aggregation = _aggregation; _tablet_reader_params.version = Version(0, _version); @@ -208,7 +219,7 @@ Status OlapScanner::_init_tablet_reader_params( _tablet_reader_params.return_columns.push_back(i); } for (auto index : _return_columns) { - if (_tablet->tablet_schema().column(index).is_key()) { + if (_tablet_schema.column(index).is_key()) { continue; } else { _tablet_reader_params.return_columns.push_back(index); @@ -217,13 +228,12 @@ Status OlapScanner::_init_tablet_reader_params( } // use _tablet_reader_params.return_columns, because reader use this to merge sort - Status res = - _read_row_cursor.init(_tablet->tablet_schema(), _tablet_reader_params.return_columns); + Status res = _read_row_cursor.init(_tablet_schema, _tablet_reader_params.return_columns); if (!res.ok()) { LOG(WARNING) << "fail to init row cursor.res = " << res; return Status::InternalError("failed to initialize storage read row cursor"); } - _read_row_cursor.allocate_memory_for_string_type(_tablet->tablet_schema()); + _read_row_cursor.allocate_memory_for_string_type(_tablet_schema); // If a agg node is this scan node direct parent // we will not call agg object finalize method in scan node, @@ -242,7 +252,9 @@ Status OlapScanner::_init_return_columns(bool need_seq_col) { if (!slot->is_materialized()) { continue; } - int32_t index = _tablet->field_index(slot->col_name()); + int32_t index = slot->col_unique_id() >= 0 + ? _tablet_schema.field_index(slot->col_unique_id()) + : _tablet_schema.field_index(slot->col_name()); if (index < 0) { std::stringstream ss; ss << "field name is invalid. field=" << slot->col_name(); @@ -250,7 +262,7 @@ Status OlapScanner::_init_return_columns(bool need_seq_col) { return Status::InternalError(ss.str()); } _return_columns.push_back(index); - if (slot->is_nullable() && !_tablet->tablet_schema().column(index).is_nullable()) + if (slot->is_nullable() && !_tablet_schema.column(index).is_nullable()) _tablet_columns_convert_to_null_set.emplace(index); _query_slots.push_back(slot); } @@ -259,13 +271,13 @@ Status OlapScanner::_init_return_columns(bool need_seq_col) { if (_tablet->tablet_schema().has_sequence_col() && need_seq_col) { bool has_replace_col = false; for (auto col : _return_columns) { - if (_tablet->tablet_schema().column(col).aggregation() == + if (_tablet_schema.column(col).aggregation() == FieldAggregationMethod::OLAP_FIELD_AGGREGATION_REPLACE) { has_replace_col = true; break; } } - if (auto sequence_col_idx = _tablet->tablet_schema().sequence_col_idx(); + if (auto sequence_col_idx = _tablet_schema.sequence_col_idx(); has_replace_col && std::find(_return_columns.begin(), _return_columns.end(), sequence_col_idx) == _return_columns.end()) { _return_columns.push_back(sequence_col_idx); diff --git a/be/src/exec/olap_scanner.h b/be/src/exec/olap_scanner.h index 65086eaceb81139..a128026217b0e10 100644 --- a/be/src/exec/olap_scanner.h +++ b/be/src/exec/olap_scanner.h @@ -145,6 +145,8 @@ class OlapScanner { MonotonicStopWatch _watcher; std::shared_ptr _mem_tracker; + + TabletSchema _tablet_schema; }; } // namespace doris diff --git a/be/src/exec/tablet_info.cpp b/be/src/exec/tablet_info.cpp index a19410c5c35522c..d572cfc71df8f5b 100644 --- a/be/src/exec/tablet_info.cpp +++ b/be/src/exec/tablet_info.cpp @@ -31,6 +31,9 @@ void OlapTableIndexSchema::to_protobuf(POlapTableIndexSchema* pindex) const { for (auto slot : slots) { pindex->add_columns(slot->col_name()); } + for (auto column : columns) { + column->to_schema_pb(pindex->add_columns_desc()); + } } Status OlapTableSchemaParam::init(const POlapTableSchemaParam& pschema) { @@ -57,6 +60,11 @@ Status OlapTableSchemaParam::init(const POlapTableSchemaParam& pschema) { } index->slots.emplace_back(it->second); } + for (auto& pcolumn_desc : p_index.columns_desc()) { + TabletColumn* tc = _obj_pool.add(new TabletColumn()); + tc->init_from_pb(pcolumn_desc); + index->columns.emplace_back(tc); + } _indexes.emplace_back(index); } @@ -90,6 +98,11 @@ Status OlapTableSchemaParam::init(const TOlapTableSchemaParam& tschema) { } index->slots.emplace_back(it->second); } + for (auto& tcolumn_desc : t_index.columns_desc) { + TabletColumn* tc = _obj_pool.add(new TabletColumn()); + tc->init_from_thrift(tcolumn_desc); + index->columns.emplace_back(tc); + } _indexes.emplace_back(index); } diff --git a/be/src/exec/tablet_info.h b/be/src/exec/tablet_info.h index 40279a70fe83c61..f287fcfa0dd9147 100644 --- a/be/src/exec/tablet_info.h +++ b/be/src/exec/tablet_info.h @@ -27,6 +27,7 @@ #include "common/status.h" #include "gen_cpp/Descriptors_types.h" #include "gen_cpp/descriptors.pb.h" +#include "olap/tablet_schema.h" #include "runtime/descriptors.h" #include "runtime/raw_value.h" #include "runtime/tuple.h" @@ -41,6 +42,7 @@ struct OlapTableIndexSchema { int64_t index_id; std::vector slots; int32_t schema_hash; + std::vector columns; void to_protobuf(POlapTableIndexSchema* pindex) const; }; diff --git a/be/src/olap/base_tablet.cpp b/be/src/olap/base_tablet.cpp index 00b7dc30ae41048..cb2e7561f2bfd3f 100644 --- a/be/src/olap/base_tablet.cpp +++ b/be/src/olap/base_tablet.cpp @@ -72,4 +72,15 @@ void BaseTablet::_gen_tablet_path() { } } +bool BaseTablet::set_tablet_schema_into_rowset_meta() { + bool flag = false; + for (RowsetMetaSharedPtr rowset_meta : _tablet_meta->all_mutable_rs_metas()) { + if (!rowset_meta->get_rowset_pb().has_tablet_schema()) { + rowset_meta->set_tablet_schema(&_schema); + flag = true; + } + } + return flag; +} + } /* namespace doris */ diff --git a/be/src/olap/base_tablet.h b/be/src/olap/base_tablet.h index 69a092b84a5b378..99c7d1cac382ee7 100644 --- a/be/src/olap/base_tablet.h +++ b/be/src/olap/base_tablet.h @@ -66,7 +66,9 @@ class BaseTablet : public std::enable_shared_from_this { } // properties encapsulated in TabletSchema - const TabletSchema& tablet_schema() const; + virtual const TabletSchema& tablet_schema() const; + + bool set_tablet_schema_into_rowset_meta(); protected: void _gen_tablet_path(); diff --git a/be/src/olap/collect_iterator.cpp b/be/src/olap/collect_iterator.cpp index cf7b079004956ff..de163eb3b085e77 100644 --- a/be/src/olap/collect_iterator.cpp +++ b/be/src/olap/collect_iterator.cpp @@ -55,9 +55,9 @@ Status CollectIterator::add_child(RowsetReaderSharedPtr rs_reader) { // then merged with the base rowset. void CollectIterator::build_heap(const std::vector& rs_readers) { DCHECK(rs_readers.size() == _children.size()); - _reverse = _reader->_tablet->tablet_schema().keys_type() == KeysType::UNIQUE_KEYS; - SortType sort_type = _reader->_tablet->tablet_schema().sort_type(); - int sort_col_num = _reader->_tablet->tablet_schema().sort_col_num(); + _reverse = _reader->_tablet_schema->keys_type() == KeysType::UNIQUE_KEYS; + SortType sort_type = _reader->_tablet_schema->sort_type(); + int sort_col_num = _reader->_tablet_schema->sort_col_num(); if (_children.empty()) { _inner_iter.reset(nullptr); return; @@ -200,7 +200,7 @@ CollectIterator::Level0Iterator::Level0Iterator(RowsetReaderSharedPtr rs_reader, CollectIterator::Level0Iterator::~Level0Iterator() = default; Status CollectIterator::Level0Iterator::init() { - RETURN_NOT_OK_LOG(_row_cursor.init(_reader->_tablet->tablet_schema(), _reader->_seek_columns), + RETURN_NOT_OK_LOG(_row_cursor.init(*_reader->_tablet_schema, _reader->_seek_columns), "failed to init row cursor"); return (this->*_refresh_current_row)(); } diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index a54329fb8a8c013..d5e9a6190cf26c6 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -17,7 +17,10 @@ #include "olap/compaction.h" +#include "common/status.h" #include "gutil/strings/substitute.h" +#include "olap/rowset/rowset_meta.h" +#include "olap/tablet.h" #include "util/time.h" #include "util/trace.h" @@ -141,8 +144,15 @@ Status Compaction::do_compaction_impl(int64_t permits) { LOG(INFO) << "start " << merge_type << compaction_name() << ". tablet=" << _tablet->full_name() << ", output_version=" << _output_version << ", permits: " << permits; - - RETURN_NOT_OK(construct_output_rowset_writer()); + // get cur schema if rowset schema exist, rowset schema must be newer than tablet schema + std::vector rowset_metas; + rowset_metas.resize(_input_rowsets.size()); + std::transform(_input_rowsets.begin(), _input_rowsets.end(), rowset_metas.begin(), + [](const RowsetSharedPtr& rowset) { return rowset->rowset_meta(); }); + auto rowset_meta = Tablet::rowset_meta_with_max_schema_version(rowset_metas); + const TabletSchema* cur_tablet_schema = rowset_meta->tablet_schema(); + + RETURN_NOT_OK(construct_output_rowset_writer(cur_tablet_schema)); RETURN_NOT_OK(construct_input_rowset_readers()); TRACE("prepare finished"); @@ -152,11 +162,11 @@ Status Compaction::do_compaction_impl(int64_t permits) { Status res; if (use_vectorized_compaction) { - res = Merger::vmerge_rowsets(_tablet, compaction_type(), _input_rs_readers, - _output_rs_writer.get(), &stats); + res = Merger::vmerge_rowsets(_tablet, compaction_type(), cur_tablet_schema, + _input_rs_readers, _output_rs_writer.get(), &stats); } else { - res = Merger::merge_rowsets(_tablet, compaction_type(), _input_rs_readers, - _output_rs_writer.get(), &stats); + res = Merger::merge_rowsets(_tablet, compaction_type(), cur_tablet_schema, + _input_rs_readers, _output_rs_writer.get(), &stats); } if (!res.ok()) { @@ -219,8 +229,8 @@ Status Compaction::do_compaction_impl(int64_t permits) { return Status::OK(); } -Status Compaction::construct_output_rowset_writer() { - return _tablet->create_rowset_writer(_output_version, VISIBLE, NONOVERLAPPING, +Status Compaction::construct_output_rowset_writer(const TabletSchema* schema) { + return _tablet->create_rowset_writer(_output_version, VISIBLE, NONOVERLAPPING, schema, _oldest_write_timestamp, _newest_write_timestamp, &_output_rs_writer); } diff --git a/be/src/olap/compaction.h b/be/src/olap/compaction.h index bd76ded20a62c48..648ff789358da5e 100644 --- a/be/src/olap/compaction.h +++ b/be/src/olap/compaction.h @@ -67,7 +67,7 @@ class Compaction { Status modify_rowsets(); void gc_output_rowset(); - Status construct_output_rowset_writer(); + Status construct_output_rowset_writer(const TabletSchema* schema); Status construct_input_rowset_readers(); Status check_version_continuity(const std::vector& rowsets); diff --git a/be/src/olap/convert_rowset.cpp b/be/src/olap/convert_rowset.cpp new file mode 100644 index 000000000000000..fe88ff0360915d4 --- /dev/null +++ b/be/src/olap/convert_rowset.cpp @@ -0,0 +1,141 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "olap/convert_rowset.h" + +namespace doris { + +Status ConvertRowset::do_convert() { + if (!_tablet->init_succeeded()) { + return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR); + } + std::unique_lock base_compaction_lock(_tablet->get_base_compaction_lock(), + std::try_to_lock); + std::unique_lock cumulative_compaction_lock( + _tablet->get_cumulative_compaction_lock(), std::try_to_lock); + if (!base_compaction_lock.owns_lock() || !cumulative_compaction_lock.owns_lock()) { + LOG(INFO) << "The tablet is under compaction. tablet=" << _tablet->full_name(); + return Status::OLAPInternalError(OLAP_ERR_CE_TRY_CE_LOCK_ERROR); + } + + std::vector alpah_rowsets; + _tablet->find_alpha_rowsets(&alpah_rowsets); + + Merger::Statistics stats; + Status res; + const size_t max_convert_row_count = 20000000; + size_t row_count = 0; + for (size_t i = 0; i < alpah_rowsets.size(); ++i) { + Version output_version = + Version(alpah_rowsets[i]->start_version(), alpah_rowsets[i]->end_version()); + + RowsetReaderSharedPtr input_rs_reader; + RETURN_NOT_OK(alpah_rowsets[i]->create_reader(&input_rs_reader)); + + std::unique_ptr output_rs_writer; + RETURN_NOT_OK(_tablet->create_rowset_writer(output_version, VISIBLE, NONOVERLAPPING, + &_tablet->tablet_schema(), &output_rs_writer)); + res = Merger::merge_rowsets(_tablet, ReaderType::READER_BASE_COMPACTION, + &_tablet->tablet_schema(), {input_rs_reader}, + output_rs_writer.get(), &stats); + + if (!res.ok()) { + LOG(WARNING) << "fail to convert rowset. res=" << res + << ", tablet=" << _tablet->full_name(); + return res; + } else { + auto output_rowset = output_rs_writer->build(); + if (output_rowset == nullptr) { + LOG(WARNING) << "rowset writer build failed" + << ", tablet=" << _tablet->full_name(); + return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR); + } + + RETURN_NOT_OK(check_correctness(alpah_rowsets[i], output_rowset, stats)); + + row_count += alpah_rowsets[i]->num_rows(); + + RETURN_NOT_OK(_modify_rowsets(alpah_rowsets[i], output_rowset)); + + LOG(INFO) << "succeed to convert rowset" + << ". tablet=" << _tablet->full_name() + << ", output_version=" << output_version + << ", disk=" << _tablet->data_dir()->path(); + + if (row_count >= max_convert_row_count) { + break; + } + } + } + return Status::OK(); +} + +Status ConvertRowset::check_correctness(RowsetSharedPtr input_rowset, RowsetSharedPtr output_rowset, + const Merger::Statistics& stats) { + // 1. check row number + if (input_rowset->num_rows() != + output_rowset->num_rows() + stats.merged_rows + stats.filtered_rows) { + LOG(WARNING) << "row_num does not match between input and output! " + << "input_row_num=" << input_rowset->num_rows() + << ", merged_row_num=" << stats.merged_rows + << ", filtered_row_num=" << stats.filtered_rows + << ", output_row_num=" << output_rowset->num_rows(); + + // ATTN(cmy): We found that the num_rows in some rowset meta may be set to the wrong value, + // but it is not known which version of the code has the problem. So when the convert + // result is inconsistent, we then try to verify by num_rows recorded in segment_groups. + // If the check passes, ignore the error and set the correct value in the output rowset meta + // to fix this problem. + // Only handle alpha rowset because we only find this bug in alpha rowset + int64_t num_rows = _get_input_num_rows_from_seg_grps(input_rowset); + if (num_rows == -1) { + return Status::OLAPInternalError(OLAP_ERR_CHECK_LINES_ERROR); + } + if (num_rows != output_rowset->num_rows() + stats.merged_rows + stats.filtered_rows) { + // If it is still incorrect, it may be another problem + LOG(WARNING) << "row_num got from seg groups does not match between cumulative input " + "and output! " + << "input_row_num=" << num_rows << ", merged_row_num=" << stats.merged_rows + << ", filtered_row_num=" << stats.filtered_rows + << ", output_row_num=" << output_rowset->num_rows(); + + return Status::OLAPInternalError(OLAP_ERR_CHECK_LINES_ERROR); + } + } + return Status::OK(); +} + +int64_t ConvertRowset::_get_input_num_rows_from_seg_grps(RowsetSharedPtr rowset) { + int64_t num_rows = 0; + for (auto& seg_grp : rowset->rowset_meta()->alpha_rowset_extra_meta_pb().segment_groups()) { + num_rows += seg_grp.num_rows(); + } + return num_rows; +} +Status ConvertRowset::_modify_rowsets(RowsetSharedPtr input_rowset, RowsetSharedPtr output_rowset) { + std::vector input_rowsets; + input_rowsets.push_back(input_rowset); + + std::vector output_rowsets; + output_rowsets.push_back(output_rowset); + + std::lock_guard wrlock(_tablet->get_header_lock()); + RETURN_NOT_OK(_tablet->modify_rowsets(output_rowsets, input_rowsets, true)); + _tablet->save_meta(); + return Status::OK(); +} +} // namespace doris \ No newline at end of file diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp index 55530420b35c0b8..df51f4e990b6c8a 100644 --- a/be/src/olap/data_dir.cpp +++ b/be/src/olap/data_dir.cpp @@ -485,6 +485,11 @@ Status DataDir::load() { << " schema hash: " << rowset_meta->tablet_schema_hash() << " for txn: " << rowset_meta->txn_id(); } + if (!rowset_meta->get_rowset_pb().has_tablet_schema()) { + rowset_meta->set_tablet_schema(&tablet->tablet_schema()); + RowsetMetaManager::save(_meta, rowset_meta->tablet_uid(), rowset_meta->rowset_id(), + rowset_meta->get_rowset_pb()); + } } else if (rowset_meta->rowset_state() == RowsetStatePB::VISIBLE && rowset_meta->tablet_uid() == tablet->tablet_uid()) { Status publish_status = tablet->add_rowset(rowset); @@ -506,6 +511,15 @@ Status DataDir::load() { ++invalid_rowset_counter; } } + + for (int64_t tablet_id : tablet_ids) { + TabletSharedPtr tablet = _tablet_manager->get_tablet(tablet_id); + if (tablet->set_tablet_schema_into_rowset_meta()) { + TabletMetaManager::save(this, tablet->tablet_id(), tablet->schema_hash(), + tablet->tablet_meta()); + } + } + // At startup, we only count these invalid rowset, but do not actually delete it. // The actual delete operation is in StorageEngine::_clean_unused_rowset_metas, // which is cleaned up uniformly by the background cleanup thread. diff --git a/be/src/olap/delta_writer.cpp b/be/src/olap/delta_writer.cpp index b088ef50cc44985..786fb6c0911724a 100644 --- a/be/src/olap/delta_writer.cpp +++ b/be/src/olap/delta_writer.cpp @@ -40,7 +40,7 @@ DeltaWriter::DeltaWriter(WriteRequest* req, StorageEngine* storage_engine, bool _tablet(nullptr), _cur_rowset(nullptr), _rowset_writer(nullptr), - _tablet_schema(nullptr), + _tablet_schema(new TabletSchema), _delta_written_success(false), _storage_engine(storage_engine), _is_vec(is_vec) {} @@ -121,10 +121,11 @@ Status DeltaWriter::init() { RETURN_NOT_OK(_storage_engine->txn_manager()->prepare_txn(_req.partition_id, _tablet, _req.txn_id, _req.load_id)); } + // build tablet schema in request level + _build_current_tablet_schema(_req.index_id, _req.ptable_schema_param, _tablet->tablet_schema()); RETURN_NOT_OK(_tablet->create_rowset_writer(_req.txn_id, _req.load_id, PREPARED, OVERLAPPING, - &_rowset_writer)); - _tablet_schema = &(_tablet->tablet_schema()); + _tablet_schema.get(), &_rowset_writer)); _schema.reset(new Schema(*_tablet_schema)); _reset_mem_table(); @@ -172,7 +173,6 @@ Status DeltaWriter::write(const RowBatch* row_batch, const std::vector& row if (_is_cancelled) { return Status::OLAPInternalError(OLAP_ERR_ALREADY_CANCELLED); } - for (const auto& row_idx : row_idxs) { _mem_table->insert(row_batch->get_row(row_idx)->get_tuple(0)); } @@ -197,7 +197,7 @@ Status DeltaWriter::write(const vectorized::Block* block, const std::vector if (_is_cancelled) { return Status::OLAPInternalError(OLAP_ERR_ALREADY_CANCELLED); } - + _mem_table->insert(block, row_idxs); if (_mem_table->need_to_agg()) { @@ -266,9 +266,9 @@ Status DeltaWriter::wait_flush() { } void DeltaWriter::_reset_mem_table() { - _mem_table.reset(new MemTable(_tablet->tablet_id(), _schema.get(), _tablet_schema, _req.slots, - _req.tuple_desc, _tablet->keys_type(), _rowset_writer.get(), - _mem_tracker, _is_vec)); + _mem_table.reset(new MemTable(_tablet->tablet_id(), _schema.get(), _tablet_schema.get(), + _req.slots, _req.tuple_desc, _tablet->keys_type(), + _rowset_writer.get(), _mem_tracker, _is_vec)); } Status DeltaWriter::close() { @@ -367,4 +367,17 @@ int64_t DeltaWriter::partition_id() const { return _req.partition_id; } +void DeltaWriter::_build_current_tablet_schema(int64_t index_id, + const POlapTableSchemaParam& ptable_schema_param, + const TabletSchema& ori_tablet_schema) { + *_tablet_schema = ori_tablet_schema; + //new tablet schame if new table + if (ptable_schema_param.indexes_size() > 0 && + ptable_schema_param.indexes(0).columns_desc_size() != 0 && + ptable_schema_param.indexes(0).columns_desc(0).unique_id() >= 0) { + _tablet_schema->build_current_tablet_schema(index_id, ptable_schema_param, + ori_tablet_schema); + } +} + } // namespace doris diff --git a/be/src/olap/delta_writer.h b/be/src/olap/delta_writer.h index 1e2e9b6ed06e8e7..2d1cb64a6ac57ab 100644 --- a/be/src/olap/delta_writer.h +++ b/be/src/olap/delta_writer.h @@ -47,6 +47,8 @@ struct WriteRequest { // slots are in order of tablet's schema const std::vector* slots; bool is_high_priority = false; + POlapTableSchemaParam ptable_schema_param; + int64_t index_id; }; // Writer for a particular (load, index, tablet). @@ -107,6 +109,10 @@ class DeltaWriter { void _reset_mem_table(); + void _build_current_tablet_schema(int64_t index_id, + const POlapTableSchemaParam& table_schema_param, + const TabletSchema& ori_tablet_schema); + bool _is_init = false; bool _is_cancelled = false; WriteRequest _req; @@ -116,7 +122,11 @@ class DeltaWriter { // TODO: Recheck the lifetime of _mem_table, Look should use unique_ptr std::shared_ptr _mem_table; std::unique_ptr _schema; - const TabletSchema* _tablet_schema; + //const TabletSchema* _tablet_schema; + // tablet schema owned by delta writer, all write will use this tablet schema + // it's build from tablet_schema(stored when create tablet) and OlapTableSchema + // every request will have it's own tablet schema so simple schema change can work + std::shared_ptr _tablet_schema; bool _delta_written_success; StorageEngine* _storage_engine; diff --git a/be/src/olap/merger.cpp b/be/src/olap/merger.cpp index c9fec74b25697d8..c9cba8ca1d4cd84 100644 --- a/be/src/olap/merger.cpp +++ b/be/src/olap/merger.cpp @@ -30,6 +30,7 @@ namespace doris { Status Merger::merge_rowsets(TabletSharedPtr tablet, ReaderType reader_type, + const TabletSchema* cur_tablet_schema, const std::vector& src_rowset_readers, RowsetWriter* dst_rowset_writer, Merger::Statistics* stats_output) { TRACE_COUNTER_SCOPE_LATENCY_US("merge_rowsets_latency_us"); @@ -40,13 +41,15 @@ Status Merger::merge_rowsets(TabletSharedPtr tablet, ReaderType reader_type, reader_params.reader_type = reader_type; reader_params.rs_readers = src_rowset_readers; reader_params.version = dst_rowset_writer->version(); + + reader_params.tablet_schema = cur_tablet_schema; RETURN_NOT_OK(reader.init(reader_params)); RowCursor row_cursor; RETURN_NOT_OK_LOG( - row_cursor.init(tablet->tablet_schema()), + row_cursor.init(*cur_tablet_schema), "failed to init row cursor when merging rowsets of tablet " + tablet->full_name()); - row_cursor.allocate_memory_for_string_type(tablet->tablet_schema()); + row_cursor.allocate_memory_for_string_type(*cur_tablet_schema); std::unique_ptr mem_pool(new MemPool("Merger:merge_rowsets")); @@ -88,6 +91,7 @@ Status Merger::merge_rowsets(TabletSharedPtr tablet, ReaderType reader_type, } Status Merger::vmerge_rowsets(TabletSharedPtr tablet, ReaderType reader_type, + const TabletSchema* cur_tablet_schema, const std::vector& src_rowset_readers, RowsetWriter* dst_rowset_writer, Statistics* stats_output) { TRACE_COUNTER_SCOPE_LATENCY_US("merge_rowsets_latency_us"); @@ -98,8 +102,9 @@ Status Merger::vmerge_rowsets(TabletSharedPtr tablet, ReaderType reader_type, reader_params.reader_type = reader_type; reader_params.rs_readers = src_rowset_readers; reader_params.version = dst_rowset_writer->version(); + reader_params.tablet_schema = cur_tablet_schema; - const auto& schema = tablet->tablet_schema(); + const auto& schema = *cur_tablet_schema; reader_params.return_columns.resize(schema.num_columns()); std::iota(reader_params.return_columns.begin(), reader_params.return_columns.end(), 0); reader_params.origin_return_columns = &reader_params.return_columns; diff --git a/be/src/olap/merger.h b/be/src/olap/merger.h index 5f258779fb56601..4d37e6ccd6b138c 100644 --- a/be/src/olap/merger.h +++ b/be/src/olap/merger.h @@ -36,10 +36,12 @@ class Merger { // return OLAP_SUCCESS and set statistics into `*stats_output`. // return others on error static Status merge_rowsets(TabletSharedPtr tablet, ReaderType reader_type, + const TabletSchema* cur_tablet_schema, const std::vector& src_rowset_readers, RowsetWriter* dst_rowset_writer, Statistics* stats_output); static Status vmerge_rowsets(TabletSharedPtr tablet, ReaderType reader_type, + const TabletSchema* cur_tablet_schema, const std::vector& src_rowset_readers, RowsetWriter* dst_rowset_writer, Statistics* stats_output); }; diff --git a/be/src/olap/push_handler.cpp b/be/src/olap/push_handler.cpp index 5b3d1b2cbc5520e..64e936739ec17bc 100644 --- a/be/src/olap/push_handler.cpp +++ b/be/src/olap/push_handler.cpp @@ -116,12 +116,16 @@ Status PushHandler::_do_streaming_ingestion(TabletSharedPtr tablet, const TPushR DeletePredicatePB del_pred; DeleteConditionHandler del_cond_handler; - { - std::shared_lock rdlock(tablet_var.tablet->get_header_lock()); - res = del_cond_handler.generate_delete_predicate( - tablet_var.tablet->tablet_schema(), request.delete_conditions, &del_pred); - del_preds.push(del_pred); + auto tablet_schema = tablet_var.tablet->tablet_schema(); + if (!request.columns_desc.empty() && request.columns_desc[0].col_unique_id >= 0) { + tablet_schema.clear_columns(); + for (const auto& column_desc : request.columns_desc) { + tablet_schema.append_column(TabletColumn(column_desc)); + } } + res = del_cond_handler.generate_delete_predicate(tablet_schema, + request.delete_conditions, &del_pred); + del_preds.push(del_pred); if (!res.ok()) { LOG(WARNING) << "fail to generate delete condition. res=" << res << ", tablet=" << tablet_var.tablet->full_name(); @@ -139,14 +143,24 @@ Status PushHandler::_do_streaming_ingestion(TabletSharedPtr tablet, const TPushR return Status::OLAPInternalError(OLAP_ERR_TOO_MANY_VERSION); } - // write + auto tablet_schema = tablet_vars->at(0).tablet->tablet_schema(); + if (!request.columns_desc.empty() && request.columns_desc[0].col_unique_id >= 0) { + tablet_schema.clear_columns(); + for (const auto& column_desc : request.columns_desc) { + tablet_schema.append_column(TabletColumn(column_desc)); + } + } + + // writes if (push_type == PUSH_NORMAL_V2) { res = _convert_v2(tablet_vars->at(0).tablet, tablet_vars->at(1).tablet, - &(tablet_vars->at(0).rowset_to_add), &(tablet_vars->at(1).rowset_to_add)); + &(tablet_vars->at(0).rowset_to_add), &(tablet_vars->at(1).rowset_to_add), + &tablet_schema); } else { res = _convert(tablet_vars->at(0).tablet, tablet_vars->at(1).tablet, - &(tablet_vars->at(0).rowset_to_add), &(tablet_vars->at(1).rowset_to_add)); + &(tablet_vars->at(0).rowset_to_add), &(tablet_vars->at(1).rowset_to_add), + &tablet_schema); } if (!res.ok()) { LOG(WARNING) << "fail to convert tmp file when realtime push. res=" << res @@ -205,7 +219,8 @@ void PushHandler::_get_tablet_infos(const std::vector& tablet_vars, } Status PushHandler::_convert_v2(TabletSharedPtr cur_tablet, TabletSharedPtr new_tablet, - RowsetSharedPtr* cur_rowset, RowsetSharedPtr* new_rowset) { + RowsetSharedPtr* cur_rowset, RowsetSharedPtr* new_rowset, + const TabletSchema* tablet_schema) { Status res = Status::OK(); uint32_t num_rows = 0; PUniqueId load_id; @@ -217,13 +232,13 @@ Status PushHandler::_convert_v2(TabletSharedPtr cur_tablet, TabletSharedPtr new_ // 1. init RowsetBuilder of cur_tablet for current push VLOG_NOTICE << "init rowset builder. tablet=" << cur_tablet->full_name() - << ", block_row_size=" << cur_tablet->num_rows_per_row_block(); + << ", block_row_size=" << tablet_schema->num_rows_per_row_block(); // although the spark load output files are fully sorted, // but it depends on thirparty implementation, so we conservatively // set this value to OVERLAP_UNKNOWN std::unique_ptr rowset_writer; res = cur_tablet->create_rowset_writer(_request.transaction_id, load_id, PREPARED, - OVERLAP_UNKNOWN, &rowset_writer); + OVERLAP_UNKNOWN, tablet_schema, &rowset_writer); if (!res.ok()) { LOG(WARNING) << "failed to init rowset writer, tablet=" << cur_tablet->full_name() << ", txn_id=" << _request.transaction_id << ", res=" << res; @@ -245,7 +260,7 @@ Status PushHandler::_convert_v2(TabletSharedPtr cur_tablet, TabletSharedPtr new_ } // init schema - std::unique_ptr schema(new (std::nothrow) Schema(cur_tablet->tablet_schema())); + std::unique_ptr schema(new (std::nothrow) Schema(*tablet_schema)); if (schema == nullptr) { LOG(WARNING) << "fail to create schema. tablet=" << cur_tablet->full_name(); res = Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR); @@ -313,8 +328,8 @@ Status PushHandler::_convert_v2(TabletSharedPtr cur_tablet, TabletSharedPtr new_ // 5. Convert data for schema change tables VLOG_TRACE << "load to related tables of schema_change if possible."; if (new_tablet != nullptr) { - res = SchemaChangeHandler::schema_version_convert(cur_tablet, new_tablet, cur_rowset, - new_rowset, *_desc_tbl); + res = SchemaChangeHandler::schema_version_convert( + cur_tablet, new_tablet, cur_rowset, new_rowset, *_desc_tbl, tablet_schema); if (!res.ok()) { LOG(WARNING) << "failed to change schema version for delta." << "[res=" << res << " new_tablet='" << new_tablet->full_name() @@ -329,7 +344,8 @@ Status PushHandler::_convert_v2(TabletSharedPtr cur_tablet, TabletSharedPtr new_ } Status PushHandler::_convert(TabletSharedPtr cur_tablet, TabletSharedPtr new_tablet, - RowsetSharedPtr* cur_rowset, RowsetSharedPtr* new_rowset) { + RowsetSharedPtr* cur_rowset, RowsetSharedPtr* new_rowset, + const TabletSchema* tablet_schema) { Status res = Status::OK(); RowCursor row; BinaryFile raw_file; @@ -376,7 +392,7 @@ Status PushHandler::_convert(TabletSharedPtr cur_tablet, TabletSharedPtr new_tab } // init BinaryReader - if (!(res = reader->init(cur_tablet, &raw_file))) { + if (!(res = reader->init(tablet_schema, &raw_file))) { LOG(WARNING) << "fail to init reader. res=" << res << ", tablet=" << cur_tablet->full_name() << ", file=" << _request.http_file_path; @@ -388,7 +404,7 @@ Status PushHandler::_convert(TabletSharedPtr cur_tablet, TabletSharedPtr new_tab // 2. init RowsetBuilder of cur_tablet for current push std::unique_ptr rowset_writer; res = cur_tablet->create_rowset_writer(_request.transaction_id, load_id, PREPARED, - OVERLAP_UNKNOWN, &rowset_writer); + OVERLAP_UNKNOWN, tablet_schema, &rowset_writer); if (!res.ok()) { LOG(WARNING) << "failed to init rowset writer, tablet=" << cur_tablet->full_name() << ", txn_id=" << _request.transaction_id << ", res=" << res; @@ -400,7 +416,7 @@ Status PushHandler::_convert(TabletSharedPtr cur_tablet, TabletSharedPtr new_tab << ", block_row_size=" << cur_tablet->num_rows_per_row_block(); // 4. Init RowCursor - if (!(res = row.init(cur_tablet->tablet_schema()))) { + if (!(res = row.init(*tablet_schema))) { LOG(WARNING) << "fail to init rowcursor. res=" << res; break; } @@ -453,8 +469,8 @@ Status PushHandler::_convert(TabletSharedPtr cur_tablet, TabletSharedPtr new_tab // 7. Convert data for schema change tables VLOG_TRACE << "load to related tables of schema_change if possible."; if (new_tablet != nullptr) { - res = SchemaChangeHandler::schema_version_convert(cur_tablet, new_tablet, cur_rowset, - new_rowset, *_desc_tbl); + res = SchemaChangeHandler::schema_version_convert( + cur_tablet, new_tablet, cur_rowset, new_rowset, *_desc_tbl, tablet_schema); if (!res.ok()) { LOG(WARNING) << "failed to change schema version for delta." << "[res=" << res << " new_tablet='" << new_tablet->full_name() @@ -500,13 +516,13 @@ IBinaryReader* IBinaryReader::create(bool need_decompress) { BinaryReader::BinaryReader() : _row_buf(nullptr), _row_buf_size(0) {} -Status BinaryReader::init(TabletSharedPtr tablet, BinaryFile* file) { +Status BinaryReader::init(const TabletSchema* tablet_schema, BinaryFile* file) { Status res = Status::OK(); do { _file = file; _content_len = _file->file_length() - _file->header_size(); - _row_buf_size = tablet->row_size(); + _row_buf_size = _tablet_schema->row_size(); _row_buf = new (std::nothrow) char[_row_buf_size]; if (_row_buf == nullptr) { @@ -521,7 +537,7 @@ Status BinaryReader::init(TabletSharedPtr tablet, BinaryFile* file) { break; } - _tablet = tablet; + _tablet_schema = tablet_schema; _ready = true; } while (false); @@ -545,10 +561,10 @@ Status BinaryReader::next(RowCursor* row) { return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR); } - const TabletSchema& schema = _tablet->tablet_schema(); + const TabletSchema& schema = *_tablet_schema; size_t offset = 0; size_t field_size = 0; - size_t num_null_bytes = (_tablet->num_null_columns() + 7) / 8; + size_t num_null_bytes = (schema.num_null_columns() + 7) / 8; if (!(res = _file->read(_row_buf + offset, num_null_bytes))) { LOG(WARNING) << "read file for one row fail. res=" << res; @@ -642,7 +658,7 @@ LzoBinaryReader::LzoBinaryReader() _row_num(0), _next_row_start(0) {} -Status LzoBinaryReader::init(TabletSharedPtr tablet, BinaryFile* file) { +Status LzoBinaryReader::init(const TabletSchema* tablet_schema, BinaryFile* file) { Status res = Status::OK(); do { @@ -663,7 +679,7 @@ Status LzoBinaryReader::init(TabletSharedPtr tablet, BinaryFile* file) { break; } - _tablet = tablet; + _tablet_schema = tablet_schema; _ready = true; } while (false); @@ -696,10 +712,10 @@ Status LzoBinaryReader::next(RowCursor* row) { } } - const TabletSchema& schema = _tablet->tablet_schema(); + const TabletSchema& schema = *_tablet_schema; size_t offset = 0; size_t field_size = 0; - size_t num_null_bytes = (_tablet->num_null_columns() + 7) / 8; + size_t num_null_bytes = (schema.num_null_columns() + 7) / 8; size_t p = 0; for (size_t i = 0; i < schema.num_columns(); ++i) { @@ -787,7 +803,7 @@ Status LzoBinaryReader::_next_block() { SAFE_DELETE_ARRAY(_row_buf); _max_row_num = _row_num; - _max_row_buf_size = _max_row_num * _tablet->row_size(); + _max_row_buf_size = _max_row_num * _tablet_schema->row_size(); _row_buf = new (std::nothrow) char[_max_row_buf_size]; if (_row_buf == nullptr) { LOG(WARNING) << "fail to malloc rows buf. size=" << _max_row_buf_size; diff --git a/be/src/olap/push_handler.h b/be/src/olap/push_handler.h index a290eb01c71be8e..70c134c9839ce44 100644 --- a/be/src/olap/push_handler.h +++ b/be/src/olap/push_handler.h @@ -60,11 +60,13 @@ class PushHandler { private: Status _convert_v2(TabletSharedPtr cur_tablet, TabletSharedPtr new_tablet_vec, - RowsetSharedPtr* cur_rowset, RowsetSharedPtr* new_rowset); + RowsetSharedPtr* cur_rowset, RowsetSharedPtr* new_rowset, + const TabletSchema* tablet_schema); // Convert local data file to internal formatted delta, // return new delta's SegmentGroup Status _convert(TabletSharedPtr cur_tablet, TabletSharedPtr new_tablet_vec, - RowsetSharedPtr* cur_rowset, RowsetSharedPtr* new_rowset); + RowsetSharedPtr* cur_rowset, RowsetSharedPtr* new_rowset, + const TabletSchema* tablet_schema); // Only for debug std::string _debug_version_list(const Versions& versions) const; @@ -112,7 +114,7 @@ class IBinaryReader { static IBinaryReader* create(bool need_decompress); virtual ~IBinaryReader() = default; - virtual Status init(TabletSharedPtr tablet, BinaryFile* file) = 0; + virtual Status init(const TabletSchema* tablet_schema, BinaryFile* file) = 0; virtual Status finalize() = 0; virtual Status next(RowCursor* row) = 0; @@ -131,7 +133,7 @@ class IBinaryReader { _ready(false) {} BinaryFile* _file; - TabletSharedPtr _tablet; + const TabletSchema* _tablet_schema; size_t _content_len; size_t _curr; uint32_t _adler_checksum; @@ -144,7 +146,7 @@ class BinaryReader : public IBinaryReader { explicit BinaryReader(); ~BinaryReader() override { finalize(); } - Status init(TabletSharedPtr tablet, BinaryFile* file) override; + Status init(const TabletSchema* tablet_schema, BinaryFile* file) override; Status finalize() override; Status next(RowCursor* row) override; @@ -161,7 +163,7 @@ class LzoBinaryReader : public IBinaryReader { explicit LzoBinaryReader(); ~LzoBinaryReader() override { finalize(); } - Status init(TabletSharedPtr tablet, BinaryFile* file) override; + Status init(const TabletSchema* tablet_schema, BinaryFile* file) override; Status finalize() override; Status next(RowCursor* row) override; diff --git a/be/src/olap/reader.cpp b/be/src/olap/reader.cpp index 3d8a3f398f6195a..a3286f4b338cacd 100644 --- a/be/src/olap/reader.cpp +++ b/be/src/olap/reader.cpp @@ -188,7 +188,7 @@ Status TabletReader::_capture_rs_readers(const ReaderParams& read_params, bool need_ordered_result = true; if (read_params.reader_type == READER_QUERY) { - if (_tablet->tablet_schema().keys_type() == DUP_KEYS) { + if (_tablet_schema->keys_type() == DUP_KEYS) { // duplicated keys are allowed, no need to merge sort keys in rowset need_ordered_result = false; } @@ -200,7 +200,7 @@ Status TabletReader::_capture_rs_readers(const ReaderParams& read_params, } _reader_context.reader_type = read_params.reader_type; - _reader_context.tablet_schema = &_tablet->tablet_schema(); + _reader_context.tablet_schema = _tablet_schema; _reader_context.need_ordered_result = need_ordered_result; _reader_context.return_columns = &_return_columns; _reader_context.seek_columns = &_seek_columns; @@ -235,6 +235,7 @@ Status TabletReader::_init_params(const ReaderParams& read_params) { _need_agg_finalize = read_params.need_agg_finalize; _reader_type = read_params.reader_type; _tablet = read_params.tablet; + _tablet_schema = read_params.tablet_schema; _init_conditions_param(read_params); _init_load_bf_columns(read_params); @@ -259,8 +260,8 @@ Status TabletReader::_init_params(const ReaderParams& read_params) { _init_seek_columns(); - if (_tablet->tablet_schema().has_sequence_col()) { - auto sequence_col_idx = _tablet->tablet_schema().sequence_col_idx(); + if (_tablet_schema->has_sequence_col()) { + auto sequence_col_idx = _tablet_schema->sequence_col_idx(); DCHECK_NE(sequence_col_idx, -1); for (auto col : _return_columns) { // query has sequence col @@ -292,16 +293,16 @@ Status TabletReader::_init_return_columns(const ReaderParams& read_params) { } } for (auto id : read_params.return_columns) { - if (_tablet->tablet_schema().column(id).is_key()) { + if (_tablet_schema->column(id).is_key()) { _key_cids.push_back(id); } else { _value_cids.push_back(id); } } } else if (read_params.return_columns.empty()) { - for (size_t i = 0; i < _tablet->tablet_schema().num_columns(); ++i) { + for (size_t i = 0; i < _tablet_schema->num_columns(); ++i) { _return_columns.push_back(i); - if (_tablet->tablet_schema().column(i).is_key()) { + if (_tablet_schema->column(i).is_key()) { _key_cids.push_back(i); } else { _value_cids.push_back(i); @@ -314,7 +315,7 @@ Status TabletReader::_init_return_columns(const ReaderParams& read_params) { !read_params.return_columns.empty()) { _return_columns = read_params.return_columns; for (auto id : read_params.return_columns) { - if (_tablet->tablet_schema().column(id).is_key()) { + if (_tablet_schema->column(id).is_key()) { _key_cids.push_back(id); } else { _value_cids.push_back(id); @@ -323,7 +324,7 @@ Status TabletReader::_init_return_columns(const ReaderParams& read_params) { } else if (read_params.reader_type == READER_CHECKSUM) { _return_columns = read_params.return_columns; for (auto id : read_params.return_columns) { - if (_tablet->tablet_schema().column(id).is_key()) { + if (_tablet_schema->column(id).is_key()) { _key_cids.push_back(id); } else { _value_cids.push_back(id); @@ -353,7 +354,7 @@ void TabletReader::_init_seek_columns() { max_key_column_count = std::max(max_key_column_count, key.field_count()); } - for (size_t i = 0; i < _tablet->tablet_schema().num_columns(); i++) { + for (size_t i = 0; i < _tablet_schema->num_columns(); i++) { if (i < max_key_column_count || column_set.find(i) != column_set.end()) { _seek_columns.push_back(i); } @@ -373,19 +374,18 @@ Status TabletReader::_init_keys_param(const ReaderParams& read_params) { std::vector(start_key_size).swap(_keys_param.start_keys); size_t scan_key_size = read_params.start_key.front().size(); - if (scan_key_size > _tablet->tablet_schema().num_columns()) { + if (scan_key_size > _tablet_schema->num_columns()) { LOG(WARNING) << "Input param are invalid. Column count is bigger than num_columns of schema. " << "column_count=" << scan_key_size - << ", schema.num_columns=" << _tablet->tablet_schema().num_columns(); + << ", schema.num_columns=" << _tablet_schema->num_columns(); return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR); } std::vector columns(scan_key_size); std::iota(columns.begin(), columns.end(), 0); - std::shared_ptr schema = - std::make_shared(_tablet->tablet_schema().columns(), columns); + std::shared_ptr schema = std::make_shared(_tablet_schema->columns(), columns); for (size_t i = 0; i < start_key_size; ++i) { if (read_params.start_key[i].size() != scan_key_size) { @@ -396,12 +396,11 @@ Status TabletReader::_init_keys_param(const ReaderParams& read_params) { } Status res = _keys_param.start_keys[i].init_scan_key( - _tablet->tablet_schema(), read_params.start_key[i].values(), schema); + *_tablet_schema, read_params.start_key[i].values(), schema); if (!res.ok()) { LOG(WARNING) << "fail to init row cursor. res = " << res; return res; } - res = _keys_param.start_keys[i].from_tuple(read_params.start_key[i]); if (!res.ok()) { LOG(WARNING) << "fail to init row cursor from Keys. res=" << res << "key_index=" << i; @@ -419,7 +418,7 @@ Status TabletReader::_init_keys_param(const ReaderParams& read_params) { return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR); } - Status res = _keys_param.end_keys[i].init_scan_key(_tablet->tablet_schema(), + Status res = _keys_param.end_keys[i].init_scan_key(*_tablet_schema, read_params.end_key[i].values(), schema); if (!res.ok()) { LOG(WARNING) << "fail to init row cursor. res = " << res; @@ -439,13 +438,12 @@ Status TabletReader::_init_keys_param(const ReaderParams& read_params) { } void TabletReader::_init_conditions_param(const ReaderParams& read_params) { - _conditions.set_tablet_schema(&_tablet->tablet_schema()); - _all_conditions.set_tablet_schema(&_tablet->tablet_schema()); + _conditions.set_tablet_schema(_tablet_schema); + _all_conditions.set_tablet_schema(_tablet_schema); for (const auto& condition : read_params.conditions) { ColumnPredicate* predicate = _parse_to_predicate(condition); if (predicate != nullptr) { - if (_tablet->tablet_schema() - .column(_tablet->field_index(condition.column_name)) + if (_tablet_schema->column(_tablet_schema->field_index(condition.column_name)) .aggregation() != FieldAggregationMethod::OLAP_FIELD_AGGREGATION_NONE) { _value_col_predicates.push_back(predicate); } else { @@ -576,11 +574,11 @@ COMPARISON_PREDICATE_CONDITION_VALUE(ge, GreaterEqualPredicate) ColumnPredicate* TabletReader::_parse_to_predicate( const std::pair>& bloom_filter) { - int32_t index = _tablet->field_index(bloom_filter.first); + int32_t index = _tablet_schema->field_index(bloom_filter.first); if (index < 0) { return nullptr; } - const TabletColumn& column = _tablet->tablet_schema().column(index); + const TabletColumn& column = _tablet_schema->column(index); return BloomFilterColumnPredicateFactory::create_column_predicate(index, bloom_filter.second, column.type()); } @@ -588,12 +586,12 @@ ColumnPredicate* TabletReader::_parse_to_predicate( ColumnPredicate* TabletReader::_parse_to_predicate(const TCondition& condition, bool opposite) const { // TODO: not equal and not in predicate is not pushed down - int32_t index = _tablet->field_index(condition.column_name); + int32_t index = _tablet_schema->field_index(condition.column_name); if (index < 0) { return nullptr; } - const TabletColumn& column = _tablet->tablet_schema().column(index); + const TabletColumn& column = _tablet_schema->column(index); ColumnPredicate* predicate = nullptr; if ((condition.condition_op == "*=" || condition.condition_op == "!*=" || @@ -795,7 +793,7 @@ void TabletReader::_init_load_bf_columns(const ReaderParams& read_params, Condit std::set* load_bf_columns) { // add all columns with condition to load_bf_columns for (const auto& cond_column : conditions->columns()) { - if (!_tablet->tablet_schema().column(cond_column.first).is_bf_column()) { + if (!_tablet_schema->column(cond_column.first).is_bf_column()) { continue; } for (const auto& cond : cond_column.second->conds()) { @@ -807,7 +805,7 @@ void TabletReader::_init_load_bf_columns(const ReaderParams& read_params, Condit } // remove columns which have same value between start_key and end_key - int min_scan_key_len = _tablet->tablet_schema().num_columns(); + int min_scan_key_len = _tablet_schema->num_columns(); for (const auto& start_key : read_params.start_key) { min_scan_key_len = std::min(min_scan_key_len, static_cast(start_key.size())); } @@ -838,7 +836,7 @@ void TabletReader::_init_load_bf_columns(const ReaderParams& read_params, Condit if (max_equal_index == -1) { return; } - FieldType type = _tablet->tablet_schema().column(max_equal_index).type(); + FieldType type = _tablet_schema->column(max_equal_index).type(); if ((type != OLAP_FIELD_TYPE_VARCHAR && type != OLAP_FIELD_TYPE_STRING) || max_equal_index + 1 > _tablet->num_short_key_columns()) { load_bf_columns->erase(max_equal_index); @@ -858,7 +856,7 @@ Status TabletReader::_init_delete_condition(const ReaderParams& read_params) { } auto delete_init = [&]() -> Status { - return _delete_handler.init(_tablet->tablet_schema(), _tablet->delete_predicates(), + return _delete_handler.init(*_tablet_schema, _tablet->delete_predicates(), read_params.version.second, this); }; diff --git a/be/src/olap/reader.h b/be/src/olap/reader.h index c0fb42ebf89942b..2ca738f8fd09727 100644 --- a/be/src/olap/reader.h +++ b/be/src/olap/reader.h @@ -27,6 +27,7 @@ #include "olap/row_cursor.h" #include "olap/rowset/rowset_reader.h" #include "olap/tablet.h" +#include "olap/tablet_schema.h" #include "util/runtime_profile.h" namespace doris { @@ -57,6 +58,7 @@ class TabletReader { // mainly include tablet, data version and fetch range. struct ReaderParams { TabletSharedPtr tablet; + const TabletSchema* tablet_schema; ReaderType reader_type = READER_QUERY; bool direct_mode = false; bool aggregation = false; @@ -171,6 +173,7 @@ class TabletReader { std::set* load_bf_columns); TabletSharedPtr tablet() { return _tablet; } + const TabletSchema& tablet_schema() { return *_tablet_schema; } std::unique_ptr _predicate_mem_pool; std::set _load_bf_columns; @@ -183,6 +186,7 @@ class TabletReader { TabletSharedPtr _tablet; RowsetReaderContext _reader_context; + const TabletSchema* _tablet_schema; KeysParam _keys_param; std::vector _is_lower_keys_included; std::vector _is_upper_keys_included; diff --git a/be/src/olap/rowset/beta_rowset.cpp b/be/src/olap/rowset/beta_rowset.cpp index d8ef290cdaed064..14fdeee317604de 100644 --- a/be/src/olap/rowset/beta_rowset.cpp +++ b/be/src/olap/rowset/beta_rowset.cpp @@ -69,7 +69,8 @@ Status BetaRowset::do_load(bool /*use_cache*/) { return Status::OK(); } -Status BetaRowset::load_segments(std::vector* segments) { +Status BetaRowset::load_segments(std::vector* segments, + const TabletSchema* read_tablet_schema) { auto fs = _rowset_meta->fs(); if (!fs) { return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED); @@ -77,7 +78,7 @@ Status BetaRowset::load_segments(std::vector* segm for (int seg_id = 0; seg_id < num_segments(); ++seg_id) { auto seg_path = segment_file_path(seg_id); std::shared_ptr segment; - auto s = segment_v2::Segment::open(fs, seg_path, seg_id, _schema, &segment); + auto s = segment_v2::Segment::open(fs, seg_path, seg_id, read_tablet_schema, &segment); if (!s.ok()) { LOG(WARNING) << "failed to open segment. " << seg_path << " under rowset " << unique_id() << " : " << s.to_string(); diff --git a/be/src/olap/rowset/beta_rowset.h b/be/src/olap/rowset/beta_rowset.h index 9ccaa7b2b903d07..e1e963f6baab3af 100644 --- a/be/src/olap/rowset/beta_rowset.h +++ b/be/src/olap/rowset/beta_rowset.h @@ -70,7 +70,8 @@ class BetaRowset : public Rowset { bool check_file_exist() override; - Status load_segments(std::vector* segments); + Status load_segments(std::vector* segments, + const TabletSchema* read_tablet_schema); protected: BetaRowset(const TabletSchema* schema, const std::string& tablet_path, diff --git a/be/src/olap/rowset/beta_rowset_reader.cpp b/be/src/olap/rowset/beta_rowset_reader.cpp index b5ff00e077d98de..5dd7c1248363b9e 100644 --- a/be/src/olap/rowset/beta_rowset_reader.cpp +++ b/be/src/olap/rowset/beta_rowset_reader.cpp @@ -86,7 +86,7 @@ Status BetaRowsetReader::init(RowsetReaderContext* read_context) { // load segments RETURN_NOT_OK(SegmentLoader::instance()->load_segments( - _rowset, &_segment_cache_handle, + _rowset, &_segment_cache_handle, read_context->tablet_schema, read_context->reader_type == ReaderType::READER_QUERY)); // create iterator for each segment diff --git a/be/src/olap/rowset/beta_rowset_writer.cpp b/be/src/olap/rowset/beta_rowset_writer.cpp index 439327ef79a3447..87a4ba8b4ca4022 100644 --- a/be/src/olap/rowset/beta_rowset_writer.cpp +++ b/be/src/olap/rowset/beta_rowset_writer.cpp @@ -88,6 +88,8 @@ Status BetaRowsetWriter::init(const RowsetWriterContext& rowset_writer_context) _rowset_meta->set_newest_write_timestamp(_context.newest_write_timestamp); } _rowset_meta->set_tablet_uid(_context.tablet_uid); + _rowset_meta->set_tablet_schema(_context.tablet_schema); + return Status::OK(); } diff --git a/be/src/olap/rowset/rowset.cpp b/be/src/olap/rowset/rowset.cpp index 55fe81ffd32f040..009ce832bf49092 100644 --- a/be/src/olap/rowset/rowset.cpp +++ b/be/src/olap/rowset/rowset.cpp @@ -23,10 +23,7 @@ namespace doris { Rowset::Rowset(const TabletSchema* schema, const std::string& tablet_path, RowsetMetaSharedPtr rowset_meta) - : _schema(schema), - _tablet_path(tablet_path), - _rowset_meta(std::move(rowset_meta)), - _refs_by_reader(0) { + : _tablet_path(tablet_path), _rowset_meta(std::move(rowset_meta)), _refs_by_reader(0) { _is_pending = !_rowset_meta->has_version(); if (_is_pending) { _is_cumulative = false; @@ -34,6 +31,8 @@ Rowset::Rowset(const TabletSchema* schema, const std::string& tablet_path, Version version = _rowset_meta->version(); _is_cumulative = version.first != version.second; } + // build schema from RowsetMeta.tablet_schema or Tablet.tablet_schema + _schema = _rowset_meta->tablet_schema() != nullptr ? _rowset_meta->tablet_schema() : schema; } Status Rowset::load(bool use_cache) { diff --git a/be/src/olap/rowset/rowset.h b/be/src/olap/rowset/rowset.h index 158848be89bcc4b..1345be0c38446a6 100644 --- a/be/src/olap/rowset/rowset.h +++ b/be/src/olap/rowset/rowset.h @@ -138,6 +138,7 @@ class Rowset : public std::enable_shared_from_this { // publish rowset to make it visible to read void make_visible(Version version); + const TabletSchema* tablet_schema() { return _schema; } // helper class to access RowsetMeta int64_t start_version() const { return rowset_meta()->version().first; } diff --git a/be/src/olap/rowset/rowset_meta.h b/be/src/olap/rowset/rowset_meta.h index e4153e234572d9f..98aff330c0f2982 100644 --- a/be/src/olap/rowset/rowset_meta.h +++ b/be/src/olap/rowset/rowset_meta.h @@ -31,6 +31,7 @@ #include "json2pb/json_to_pb.h" #include "json2pb/pb_to_json.h" #include "olap/olap_common.h" +#include "olap/tablet_schema.h" namespace doris { @@ -313,6 +314,14 @@ class RowsetMeta { int64_t oldest_write_timestamp() const { return _rowset_meta_pb.oldest_write_timestamp(); } int64_t newest_write_timestamp() const { return _rowset_meta_pb.newest_write_timestamp(); } + void set_tablet_schema(const TabletSchema* tablet_schema) { + TabletSchemaPB* ts_pb = _rowset_meta_pb.mutable_tablet_schema(); + tablet_schema->to_schema_pb(ts_pb); + CHECK(_schema == nullptr); + _schema = std::make_shared(*tablet_schema); + } + + const TabletSchema* tablet_schema() { return _schema.get(); } private: friend class AlphaRowsetMeta; @@ -356,6 +365,10 @@ class RowsetMeta { } set_num_segments(num_segments); } + if (_rowset_meta_pb.has_tablet_schema()) { + _schema = std::make_shared(); + _schema->init_from_pb(_rowset_meta_pb.tablet_schema()); + } } friend bool operator==(const RowsetMeta& a, const RowsetMeta& b) { @@ -371,6 +384,7 @@ class RowsetMeta { private: RowsetMetaPB _rowset_meta_pb; + std::shared_ptr _schema = nullptr; RowsetId _rowset_id; io::FileSystemPtr _fs; bool _is_removed_from_rowset_meta = false; diff --git a/be/src/olap/rowset/segment_v2/segment.cpp b/be/src/olap/rowset/segment_v2/segment.cpp index fcf1d58da678c47..6d9ff92c39b80da 100644 --- a/be/src/olap/rowset/segment_v2/segment.cpp +++ b/be/src/olap/rowset/segment_v2/segment.cpp @@ -47,9 +47,8 @@ Status Segment::open(io::FileSystem* fs, const std::string& path, uint32_t segme Segment::Segment(io::FileSystem* fs, const std::string& path, uint32_t segment_id, const TabletSchema* tablet_schema) - : _fs(fs), _path(path), _segment_id(segment_id), _tablet_schema(tablet_schema) { -#ifndef BE_TEST - _mem_tracker = StorageEngine::instance()->tablet_mem_tracker(); + : _fs(fs), _path(path), _segment_id(segment_id), _tablet_schema(*tablet_schema) { +#ifndef BE_TEST _mem_tracker = StorageEngine::instance()->tablet_mem_tracker(); #else _mem_tracker = MemTracker::get_process_tracker(); #endif @@ -177,16 +176,16 @@ Status Segment::_create_column_readers() { _column_id_to_footer_ordinal.emplace(column_pb.unique_id(), ordinal); } - _column_readers.resize(_tablet_schema->columns().size()); - for (uint32_t ordinal = 0; ordinal < _tablet_schema->num_columns(); ++ordinal) { - auto& column = _tablet_schema->columns()[ordinal]; + _column_readers.resize(_tablet_schema.columns().size()); + for (uint32_t ordinal = 0; ordinal < _tablet_schema.num_columns(); ++ordinal) { + auto& column = _tablet_schema.columns()[ordinal]; auto iter = _column_id_to_footer_ordinal.find(column.unique_id()); if (iter == _column_id_to_footer_ordinal.end()) { continue; } ColumnReaderOptions opts; - opts.kept_in_memory = _tablet_schema->is_in_memory(); + opts.kept_in_memory = _tablet_schema.is_in_memory(); std::unique_ptr reader; RETURN_IF_ERROR(ColumnReader::create(opts, _footer.columns(iter->second), _footer.num_rows(), _fs, _path, &reader)); @@ -197,7 +196,7 @@ Status Segment::_create_column_readers() { Status Segment::new_column_iterator(uint32_t cid, ColumnIterator** iter) { if (_column_readers[cid] == nullptr) { - const TabletColumn& tablet_column = _tablet_schema->column(cid); + const TabletColumn& tablet_column = _tablet_schema.column(cid); if (!tablet_column.has_default_value() && !tablet_column.is_nullable()) { return Status::InternalError("invalid nonexistent column without default value."); } diff --git a/be/src/olap/rowset/segment_v2/segment.h b/be/src/olap/rowset/segment_v2/segment.h index 11925e35e95854b..2c9cb83df19f851 100644 --- a/be/src/olap/rowset/segment_v2/segment.h +++ b/be/src/olap/rowset/segment_v2/segment.h @@ -76,7 +76,7 @@ class Segment : public std::enable_shared_from_this { Status new_bitmap_index_iterator(uint32_t cid, BitmapIndexIterator** iter); - size_t num_short_keys() const { return _tablet_schema->num_short_key_columns(); } + size_t num_short_keys() const { return _tablet_schema.num_short_key_columns(); } uint32_t num_rows_per_block() const { DCHECK(_load_index_once.has_called() && _load_index_once.stored_result().ok()); @@ -121,7 +121,7 @@ class Segment : public std::enable_shared_from_this { std::string _path; uint32_t _segment_id; - const TabletSchema* _tablet_schema; + TabletSchema _tablet_schema; // This mem tracker is only for tracking memory use by segment meta data such as footer or index page. // The memory consumed by querying is tracked in segment iterator. diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp index d1a70933c994d5e..c03d3582ffa2670 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -143,7 +143,7 @@ Status SegmentIterator::_init(bool is_vec) { RETURN_IF_ERROR(_init_return_column_iterators()); RETURN_IF_ERROR(_init_bitmap_index_iterators()); // z-order can not use prefix index - if (_segment->_tablet_schema->sort_type() != SortType::ZORDER) { + if (_segment->_tablet_schema.sort_type() != SortType::ZORDER) { RETURN_IF_ERROR(_get_row_ranges_by_keys()); } RETURN_IF_ERROR(_get_row_ranges_by_column_conditions()); diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp index 7f282d05173f509..ba0609e03e37cc7 100644 --- a/be/src/olap/schema_change.cpp +++ b/be/src/olap/schema_change.cpp @@ -1632,8 +1632,8 @@ bool SchemaChangeWithSorting::_internal_sorting( std::unique_ptr rowset_writer; if (!new_tablet->create_rowset_writer(version, VISIBLE, segments_overlap, - oldest_write_timestamp, newest_write_timestamp, - &rowset_writer)) { + &new_tablet->tablet_schema(), oldest_write_timestamp, + newest_write_timestamp, &rowset_writer)) { return false; } @@ -1658,9 +1658,9 @@ Status VSchemaChangeWithSorting::_internal_sorting( MultiBlockMerger merger(new_tablet); std::unique_ptr rowset_writer; - RETURN_IF_ERROR(new_tablet->create_rowset_writer(version, VISIBLE, segments_overlap, - oldest_write_timestamp, newest_write_timestamp, - &rowset_writer)); + RETURN_IF_ERROR(new_tablet->create_rowset_writer( + version, VISIBLE, segments_overlap, &new_tablet->tablet_schema(), + oldest_write_timestamp, newest_write_timestamp, &rowset_writer)); Defer defer {[&]() { new_tablet->data_dir()->remove_pending_ids(ROWSET_ID_PREFIX + @@ -1687,10 +1687,16 @@ bool SchemaChangeWithSorting::_external_sorting(vector& src_row } rs_readers.push_back(rs_reader); } + // get cur schema if rowset schema exist, rowset schema must be newer than tablet schema + auto max_version_rowset = src_rowsets.back(); + const TabletSchema* cur_tablet_schema = max_version_rowset->rowset_meta()->tablet_schema(); + if (cur_tablet_schema == nullptr) { + cur_tablet_schema = &(new_tablet->tablet_schema()); + } Merger::Statistics stats; - auto res = Merger::merge_rowsets(new_tablet, READER_ALTER_TABLE, rs_readers, rowset_writer, - &stats); + auto res = Merger::merge_rowsets(new_tablet, READER_ALTER_TABLE, cur_tablet_schema, rs_readers, + rowset_writer, &stats); if (!res) { LOG(WARNING) << "failed to merge rowsets. tablet=" << new_tablet->full_name() << ", version=" << rowset_writer->version().first << "-" @@ -1713,8 +1719,9 @@ Status VSchemaChangeWithSorting::_external_sorting(vector& src_ } Merger::Statistics stats; - RETURN_IF_ERROR(Merger::vmerge_rowsets(new_tablet, READER_ALTER_TABLE, rs_readers, - rowset_writer, &stats)); + RETURN_IF_ERROR(Merger::vmerge_rowsets(new_tablet, READER_ALTER_TABLE, + &new_tablet->tablet_schema(), rs_readers, rowset_writer, + &stats)); _add_merged_rows(stats.merged_rows); _add_filtered_rows(stats.filtered_rows); @@ -1801,6 +1808,13 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2& // delete handlers for new tablet DeleteHandler delete_handler; std::vector return_columns; + auto base_tablet_schema = base_tablet->tablet_schema(); + if (!request.columns.empty() && request.columns[0].col_unique_id >= 0) { + base_tablet_schema.clear_columns(); + for (const auto& column : request.columns) { + base_tablet_schema.append_column(TabletColumn(column)); + } + } // begin to find deltas to convert from base tablet to new tablet so that // obtain base tablet and new tablet's push lock and header write lock to prevent loading data @@ -1811,7 +1825,7 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2& std::lock_guard new_tablet_wlock(new_tablet->get_header_lock()); // check if the tablet has alter task // if it has alter task, it means it is under old alter process - size_t num_cols = base_tablet->tablet_schema().num_columns(); + size_t num_cols = base_tablet_schema.num_columns(); return_columns.resize(num_cols); for (int i = 0; i < num_cols; ++i) { return_columns[i] = i; @@ -1821,7 +1835,7 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2& // with rs_readers RowsetReaderContext reader_context; reader_context.reader_type = READER_ALTER_TABLE; - reader_context.tablet_schema = &base_tablet->tablet_schema(); + reader_context.tablet_schema = &base_tablet_schema; reader_context.need_ordered_result = true; reader_context.delete_handler = &delete_handler; reader_context.return_columns = &return_columns; @@ -1907,7 +1921,8 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2& reader_params.tablet = base_tablet; reader_params.reader_type = READER_ALTER_TABLE; reader_params.rs_readers = rs_readers; - const auto& schema = base_tablet->tablet_schema(); + reader_params.tablet_schema = &base_tablet_schema; + const auto& schema = base_tablet_schema; reader_params.return_columns.resize(schema.num_columns()); std::iota(reader_params.return_columns.begin(), reader_params.return_columns.end(), 0); reader_params.origin_return_columns = &reader_params.return_columns; @@ -1915,8 +1930,8 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2& // BlockReader::init will call base_tablet->get_header_lock(), but this lock we already get at outer layer, so we just call TabletReader::init RETURN_NOT_OK(reader.TabletReader::init(reader_params)); - res = delete_handler.init(base_tablet->tablet_schema(), - base_tablet->delete_predicates(), end_version, &reader); + res = delete_handler.init(base_tablet_schema, base_tablet->delete_predicates(), + end_version, &reader); if (!res) { LOG(WARNING) << "init delete handler failed. base_tablet=" << base_tablet->full_name() << ", end_version=" << end_version; @@ -1947,6 +1962,7 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2& sc_params.new_tablet = new_tablet; sc_params.ref_rowset_readers = rs_readers; sc_params.delete_handler = &delete_handler; + sc_params.base_tablet_schema = &base_tablet_schema; if (request.__isset.materialized_view_params) { for (auto item : request.materialized_view_params) { AlterMaterializedViewParam mv_param; @@ -2025,7 +2041,8 @@ Status SchemaChangeHandler::schema_version_convert(TabletSharedPtr base_tablet, TabletSharedPtr new_tablet, RowsetSharedPtr* base_rowset, RowsetSharedPtr* new_rowset, - DescriptorTbl desc_tbl) { + DescriptorTbl desc_tbl, + const TabletSchema* base_schema_change) { Status res = Status::OK(); LOG(INFO) << "begin to convert delta version for schema changing. " << "base_tablet=" << base_tablet->full_name() @@ -2039,7 +2056,7 @@ Status SchemaChangeHandler::schema_version_convert(TabletSharedPtr base_tablet, const std::unordered_map materialized_function_map; if (res = _parse_request(base_tablet, new_tablet, &rb_changer, &sc_sorting, &sc_directly, - materialized_function_map, desc_tbl); + materialized_function_map, desc_tbl, base_schema_change); !res) { LOG(WARNING) << "failed to parse the request. res=" << res; return res; @@ -2053,7 +2070,7 @@ Status SchemaChangeHandler::schema_version_convert(TabletSharedPtr base_tablet, // c. Convert data DeleteHandler delete_handler; std::vector return_columns; - size_t num_cols = base_tablet->tablet_schema().num_columns(); + size_t num_cols = base_schema_change->num_columns(); return_columns.resize(num_cols); for (int i = 0; i < num_cols; ++i) { return_columns[i] = i; @@ -2061,7 +2078,7 @@ Status SchemaChangeHandler::schema_version_convert(TabletSharedPtr base_tablet, RowsetReaderContext reader_context; reader_context.reader_type = READER_ALTER_TABLE; - reader_context.tablet_schema = &base_tablet->tablet_schema(); + reader_context.tablet_schema = base_schema_change; reader_context.need_ordered_result = true; reader_context.delete_handler = &delete_handler; reader_context.return_columns = &return_columns; @@ -2077,9 +2094,10 @@ Status SchemaChangeHandler::schema_version_convert(TabletSharedPtr base_tablet, load_id.set_hi((*base_rowset)->load_id().hi()); load_id.set_lo((*base_rowset)->load_id().lo()); std::unique_ptr rowset_writer; - RETURN_NOT_OK(new_tablet->create_rowset_writer( - (*base_rowset)->txn_id(), load_id, PREPARED, - (*base_rowset)->rowset_meta()->segments_overlap(), &rowset_writer)); + RETURN_NOT_OK( + new_tablet->create_rowset_writer((*base_rowset)->txn_id(), load_id, PREPARED, + (*base_rowset)->rowset_meta()->segments_overlap(), + reader_context.tablet_schema, &rowset_writer)); auto schema_version_convert_error = [&]() -> Status { if (*new_rowset != nullptr) { @@ -2160,9 +2178,9 @@ Status SchemaChangeHandler::_convert_historical_rowsets(const SchemaChangeParams bool sc_directly = false; // a.Parse the Alter request and convert it into an internal representation - Status res = - _parse_request(sc_params.base_tablet, sc_params.new_tablet, &rb_changer, &sc_sorting, - &sc_directly, sc_params.materialized_params_map, *sc_params.desc_tbl); + Status res = _parse_request(sc_params.base_tablet, sc_params.new_tablet, &rb_changer, + &sc_sorting, &sc_directly, sc_params.materialized_params_map, + *sc_params.desc_tbl, sc_params.base_tablet_schema); auto process_alter_exit = [&]() -> Status { { @@ -2204,8 +2222,8 @@ Status SchemaChangeHandler::_convert_historical_rowsets(const SchemaChangeParams Status status = new_tablet->create_rowset_writer( rs_reader->version(), VISIBLE, rs_reader->rowset()->rowset_meta()->segments_overlap(), - rs_reader->oldest_write_timestamp(), rs_reader->newest_write_timestamp(), - &rowset_writer); + &new_tablet->tablet_schema(), rs_reader->oldest_write_timestamp(), + rs_reader->newest_write_timestamp(), &rowset_writer); if (!status.ok()) { res = Status::OLAPInternalError(OLAP_ERR_ROWSET_BUILDER_INIT); return process_alter_exit(); @@ -2267,7 +2285,7 @@ Status SchemaChangeHandler::_parse_request( bool* sc_sorting, bool* sc_directly, const std::unordered_map& materialized_function_map, - DescriptorTbl desc_tbl) { + DescriptorTbl desc_tbl, const TabletSchema* base_tablet_schema) { // set column mapping for (int i = 0, new_schema_size = new_tablet->tablet_schema().num_columns(); i < new_schema_size; ++i) { @@ -2276,7 +2294,7 @@ Status SchemaChangeHandler::_parse_request( ColumnMapping* column_mapping = rb_changer->get_mutable_column_mapping(i); if (new_column.has_reference_column()) { - int32_t column_index = base_tablet->field_index(new_column.referenced_column()); + int32_t column_index = base_tablet_schema->field_index(new_column.referenced_column()); if (column_index < 0) { LOG(WARNING) << "referenced column was missing. " @@ -2295,7 +2313,7 @@ Status SchemaChangeHandler::_parse_request( auto mvParam = materialized_function_map.find(column_name)->second; column_mapping->materialized_function = mvParam.mv_expr; column_mapping->expr = mvParam.expr; - int32_t column_index = base_tablet->field_index(mvParam.origin_column_name); + int32_t column_index = base_tablet_schema->field_index(mvParam.origin_column_name); if (column_index >= 0) { column_mapping->ref_column = column_index; continue; @@ -2307,7 +2325,7 @@ Status SchemaChangeHandler::_parse_request( } } - int32_t column_index = base_tablet->field_index(column_name); + int32_t column_index = base_tablet_schema->field_index(column_name); if (column_index >= 0) { column_mapping->ref_column = column_index; continue; @@ -2316,7 +2334,7 @@ Status SchemaChangeHandler::_parse_request( // Newly added column go here column_mapping->ref_column = -1; - if (i < base_tablet->num_short_key_columns()) { + if (i < base_tablet_schema->num_short_key_columns()) { *sc_directly = true; } RETURN_IF_ERROR( @@ -2345,7 +2363,7 @@ Status SchemaChangeHandler::_parse_request( } } - const TabletSchema& ref_tablet_schema = base_tablet->tablet_schema(); + const TabletSchema& ref_tablet_schema = *base_tablet_schema; const TabletSchema& new_tablet_schema = new_tablet->tablet_schema(); if (ref_tablet_schema.keys_type() != new_tablet_schema.keys_type()) { // only when base table is dup and mv is agg @@ -2362,7 +2380,7 @@ Status SchemaChangeHandler::_parse_request( // old keys: A B C D // new keys: A B if (new_tablet_schema.keys_type() != KeysType::DUP_KEYS && - new_tablet->num_key_columns() < base_tablet->num_key_columns()) { + new_tablet->num_key_columns() < base_tablet_schema->num_key_columns()) { // this is a table with aggregate key type, and num of key columns in new schema // is less, which means the data in new tablet should be more aggregated. // so we use sorting schema change to sort and merge the data. @@ -2370,7 +2388,7 @@ Status SchemaChangeHandler::_parse_request( return Status::OK(); } - if (base_tablet->num_short_key_columns() != new_tablet->num_short_key_columns()) { + if (base_tablet_schema->num_short_key_columns() != new_tablet->num_short_key_columns()) { // the number of short_keys changed, can't do linked schema change *sc_directly = true; return Status::OK(); diff --git a/be/src/olap/schema_change.h b/be/src/olap/schema_change.h index 4e186cae151e724..42909d459ae41cd 100644 --- a/be/src/olap/schema_change.h +++ b/be/src/olap/schema_change.h @@ -251,7 +251,8 @@ class SchemaChangeHandler { public: static Status schema_version_convert(TabletSharedPtr base_tablet, TabletSharedPtr new_tablet, RowsetSharedPtr* base_rowset, RowsetSharedPtr* new_rowset, - DescriptorTbl desc_tbl); + DescriptorTbl desc_tbl, + const TabletSchema* base_schema_change); // schema change v2, it will not set alter task in base tablet static Status process_alter_tablet_v2(const TAlterTabletReqV2& request); @@ -305,6 +306,7 @@ class SchemaChangeHandler { AlterTabletType alter_tablet_type; TabletSharedPtr base_tablet; TabletSharedPtr new_tablet; + TabletSchema* base_tablet_schema = nullptr; std::vector ref_rowset_readers; DeleteHandler* delete_handler = nullptr; std::unordered_map materialized_params_map; @@ -323,7 +325,7 @@ class SchemaChangeHandler { RowBlockChanger* rb_changer, bool* sc_sorting, bool* sc_directly, const std::unordered_map& materialized_function_map, - DescriptorTbl desc_tbl); + DescriptorTbl desc_tbl, const TabletSchema* base_tablet_schema); // Initialization Settings for creating a default value static Status _init_column_mapping(ColumnMapping* column_mapping, diff --git a/be/src/olap/segment_loader.cpp b/be/src/olap/segment_loader.cpp index a105336f6a5309b..6741fcf5a3376c4 100644 --- a/be/src/olap/segment_loader.cpp +++ b/be/src/olap/segment_loader.cpp @@ -18,6 +18,7 @@ #include "olap/segment_loader.h" #include "olap/rowset/rowset.h" +#include "olap/tablet_schema.h" #include "util/stopwatch.hpp" namespace doris { @@ -58,16 +59,17 @@ void SegmentLoader::_insert(const SegmentLoader::CacheKey& key, SegmentLoader::C } Status SegmentLoader::load_segments(const BetaRowsetSharedPtr& rowset, - SegmentCacheHandle* cache_handle, bool use_cache) { - SegmentLoader::CacheKey cache_key(rowset->rowset_id()); - if (_lookup(cache_key, cache_handle)) { + SegmentCacheHandle* cache_handle, + const TabletSchema* read_tablet_schema, bool use_cache) { + SegmentLoader::CacheKey cache_key(rowset->rowset_id(), *read_tablet_schema); + if (use_cache && _lookup(cache_key, cache_handle)) { cache_handle->owned = false; return Status::OK(); } cache_handle->owned = !use_cache; std::vector segments; - RETURN_NOT_OK(rowset->load_segments(&segments)); + RETURN_NOT_OK(rowset->load_segments(&segments, read_tablet_schema)); if (use_cache) { // memory of SegmentLoader::CacheValue will be handled by SegmentLoader diff --git a/be/src/olap/segment_loader.h b/be/src/olap/segment_loader.h index 8541ae6ed36b205..293231428508751 100644 --- a/be/src/olap/segment_loader.h +++ b/be/src/olap/segment_loader.h @@ -25,6 +25,7 @@ #include "olap/lru_cache.h" #include "olap/olap_common.h" // for rowset id #include "olap/rowset/beta_rowset.h" +#include "olap/tablet_schema.h" #include "util/time.h" namespace doris { @@ -48,11 +49,17 @@ class SegmentLoader { public: // The cache key or segment lru cache struct CacheKey { - CacheKey(RowsetId rowset_id_) : rowset_id(rowset_id_) {} + CacheKey(RowsetId rowset_id_, const TabletSchema& tablet_schema) + : rowset_id(rowset_id_), tablet_schema(tablet_schema) {} RowsetId rowset_id; + TabletSchema tablet_schema; // Encode to a flat binary which can be used as LRUCache's key - std::string encode() const { return rowset_id.to_string(); } + std::string encode() const { + TabletSchemaPB tablet_schema_pb; + tablet_schema.to_schema_pb(&tablet_schema_pb); + return rowset_id.to_string() + tablet_schema_pb.SerializeAsString(); + } }; // The cache value of segment lru cache. @@ -82,7 +89,7 @@ class SegmentLoader { // Load segments of "rowset", return the "cache_handle" which contains segments. // If use_cache is true, it will be loaded from _cache. Status load_segments(const BetaRowsetSharedPtr& rowset, SegmentCacheHandle* cache_handle, - bool use_cache = false); + const TabletSchema* read_tablet_schema, bool use_cache = false); // Try to prune the segment cache if expired. Status prune(); diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 49c07adc0c09c81..768951bccd0a00e 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -31,13 +31,16 @@ #include #include #include +<<<<<<< HEAD #include - #include "common/config.h" #include "common/logging.h" #include "common/status.h" #include "io/fs/path.h" #include "io/fs/remote_file_system.h" +======= + +>>>>>>> d9a9c0d79 ([Schema Change] support fast add/drop column (#49)) #include "olap/base_compaction.h" #include "olap/cumulative_compaction.h" #include "olap/olap_common.h" @@ -60,1741 +63,1790 @@ namespace doris { -using std::pair; -using std::nothrow; -using std::sort; -using std::string; -using std::vector; - -DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(flush_bytes, MetricUnit::BYTES); -DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(flush_count, MetricUnit::OPERATIONS); - -TabletSharedPtr Tablet::create_tablet_from_meta(TabletMetaSharedPtr tablet_meta, - const StorageParamPB& storage_param, - DataDir* data_dir) { - return std::make_shared(tablet_meta, storage_param, data_dir); -} - -Tablet::Tablet(TabletMetaSharedPtr tablet_meta, const StorageParamPB& storage_param, - DataDir* data_dir, const std::string& cumulative_compaction_type) - : BaseTablet(tablet_meta, storage_param, data_dir), - _is_bad(false), - _last_cumu_compaction_failure_millis(0), - _last_base_compaction_failure_millis(0), - _last_cumu_compaction_success_millis(0), - _last_base_compaction_success_millis(0), - _cumulative_point(K_INVALID_CUMULATIVE_POINT), - _newly_created_rowset_num(0), - _last_checkpoint_time(0), - _cumulative_compaction_type(cumulative_compaction_type), - _last_record_scan_count(0), - _last_record_scan_count_timestamp(time(nullptr)), - _is_clone_occurred(false), - _last_missed_version(-1), - _last_missed_time_s(0) { - // construct _timestamped_versioned_tracker from rs and stale rs meta - _timestamped_version_tracker.construct_versioned_tracker(_tablet_meta->all_rs_metas(), - _tablet_meta->all_stale_rs_metas()); - - INT_COUNTER_METRIC_REGISTER(_metric_entity, flush_bytes); - INT_COUNTER_METRIC_REGISTER(_metric_entity, flush_count); -} - -Status Tablet::_init_once_action() { - Status res = Status::OK(); - VLOG_NOTICE << "begin to load tablet. tablet=" << full_name() - << ", version_size=" << _tablet_meta->version_count(); + using std::pair; + using std::nothrow; + using std::sort; + using std::string; + using std::vector; + + DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(flush_bytes, MetricUnit::BYTES); + DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(flush_count, MetricUnit::OPERATIONS); + + TabletSharedPtr Tablet::create_tablet_from_meta(TabletMetaSharedPtr tablet_meta, + const StorageParamPB& storage_param, + DataDir* data_dir) { + return std::make_shared(tablet_meta, storage_param, data_dir); + } + + Tablet::Tablet(TabletMetaSharedPtr tablet_meta, const StorageParamPB& storage_param, + DataDir* data_dir, const std::string& cumulative_compaction_type) + : BaseTablet(tablet_meta, storage_param, data_dir), + _is_bad(false), + _last_cumu_compaction_failure_millis(0), + _last_base_compaction_failure_millis(0), + _last_cumu_compaction_success_millis(0), + _last_base_compaction_success_millis(0), + _cumulative_point(K_INVALID_CUMULATIVE_POINT), + _newly_created_rowset_num(0), + _last_checkpoint_time(0), + _cumulative_compaction_type(cumulative_compaction_type), + _last_record_scan_count(0), + _last_record_scan_count_timestamp(time(nullptr)), + _is_clone_occurred(false), + _last_missed_version(-1), + _last_missed_time_s(0) { + // construct _timestamped_versioned_tracker from rs and stale rs meta + _timestamped_version_tracker.construct_versioned_tracker( + _tablet_meta->all_rs_metas(), _tablet_meta->all_stale_rs_metas()); + + INT_COUNTER_METRIC_REGISTER(_metric_entity, flush_bytes); + INT_COUNTER_METRIC_REGISTER(_metric_entity, flush_count); + } + + Status Tablet::_init_once_action() { + Status res = Status::OK(); + VLOG_NOTICE << "begin to load tablet. tablet=" << full_name() + << ", version_size=" << _tablet_meta->version_count(); #ifdef BE_TEST - // init cumulative compaction policy by type - _cumulative_compaction_policy = - CumulativeCompactionPolicyFactory::create_cumulative_compaction_policy( - _cumulative_compaction_type); + // init cumulative compaction policy by type + _cumulative_compaction_policy = + CumulativeCompactionPolicyFactory::create_cumulative_compaction_policy( + _cumulative_compaction_type); #endif - for (const auto& rs_meta : _tablet_meta->all_rs_metas()) { - Version version = rs_meta->version(); - RowsetSharedPtr rowset; - res = RowsetFactory::create_rowset(&_schema, _tablet_path, rs_meta, &rowset); - if (!res.ok()) { - LOG(WARNING) << "fail to init rowset. tablet_id=" << tablet_id() - << ", schema_hash=" << schema_hash() << ", version=" << version - << ", res=" << res; - return res; + for (const auto& rs_meta : _tablet_meta->all_rs_metas()) { + Version version = rs_meta->version(); + RowsetSharedPtr rowset; + res = RowsetFactory::create_rowset(&_schema, _tablet_path, rs_meta, &rowset); + if (!res.ok()) { + LOG(WARNING) << "fail to init rowset. tablet_id=" << tablet_id() + << ", schema_hash=" << schema_hash() << ", version=" << version + << ", res=" << res; + return res; + } + _rs_version_map[version] = std::move(rowset); + } + + // init stale rowset + for (auto& stale_rs_meta : _tablet_meta->all_stale_rs_metas()) { + Version version = stale_rs_meta->version(); + RowsetSharedPtr rowset; + res = RowsetFactory::create_rowset(&_schema, _tablet_path, stale_rs_meta, &rowset); + if (!res.ok()) { + LOG(WARNING) << "fail to init stale rowset. tablet_id:" << tablet_id() + << ", schema_hash:" << schema_hash() << ", version=" << version + << ", res:" << res; + return res; + } + _stale_rs_version_map[version] = std::move(rowset); } - _rs_version_map[version] = std::move(rowset); - } - // init stale rowset - for (auto& stale_rs_meta : _tablet_meta->all_stale_rs_metas()) { - Version version = stale_rs_meta->version(); - RowsetSharedPtr rowset; - res = RowsetFactory::create_rowset(&_schema, _tablet_path, stale_rs_meta, &rowset); - if (!res.ok()) { - LOG(WARNING) << "fail to init stale rowset. tablet_id:" << tablet_id() - << ", schema_hash:" << schema_hash() << ", version=" << version - << ", res:" << res; - return res; - } - _stale_rs_version_map[version] = std::move(rowset); + return res; } - return res; -} - -Status Tablet::init() { - return _init_once.call([this] { return _init_once_action(); }); -} + Status Tablet::init() { + return _init_once.call([this] { return _init_once_action(); }); + } -// should save tablet meta to remote meta store -// if it's a primary replica -void Tablet::save_meta() { - auto res = _tablet_meta->save_meta(_data_dir); - CHECK_EQ(res, Status::OK()) << "fail to save tablet_meta. res=" << res - << ", root=" << _data_dir->path(); -} + // should save tablet meta to remote meta store + // if it's a primary replica + void Tablet::save_meta() { + auto res = _tablet_meta->save_meta(_data_dir); + CHECK_EQ(res, Status::OK()) + << "fail to save tablet_meta. res=" << res << ", root=" << _data_dir->path(); + } -Status Tablet::revise_tablet_meta(const std::vector& rowsets_to_clone, - const std::vector& versions_to_delete) { - LOG(INFO) << "begin to revise tablet. tablet=" << full_name() - << ", rowsets_to_clone=" << rowsets_to_clone.size() - << ", versions_to_delete=" << versions_to_delete.size(); - Status res = Status::OK(); - do { - // load new local tablet_meta to operate on - TabletMetaSharedPtr new_tablet_meta(new (nothrow) TabletMeta(*_tablet_meta)); + Status Tablet::revise_tablet_meta(const std::vector& rowsets_to_clone, + const std::vector& versions_to_delete) { + LOG(INFO) << "begin to revise tablet. tablet=" << full_name() + << ", rowsets_to_clone=" << rowsets_to_clone.size() + << ", versions_to_delete=" << versions_to_delete.size(); + Status res = Status::OK(); + do { + // load new local tablet_meta to operate on + TabletMetaSharedPtr new_tablet_meta(new (nothrow) TabletMeta(*_tablet_meta)); + + // delete versions from new local tablet_meta + for (const Version& version : versions_to_delete) { + new_tablet_meta->delete_rs_meta_by_version(version, nullptr); + if (new_tablet_meta->version_for_delete_predicate(version)) { + new_tablet_meta->remove_delete_predicate_by_version(version); + } + LOG(INFO) << "delete version from new local tablet_meta when clone. [table=" + << full_name() << ", version=" << version << "]"; + } - // delete versions from new local tablet_meta - for (const Version& version : versions_to_delete) { - new_tablet_meta->delete_rs_meta_by_version(version, nullptr); - if (new_tablet_meta->version_for_delete_predicate(version)) { - new_tablet_meta->remove_delete_predicate_by_version(version); + // add new cloned rowset + for (auto& rs_meta : rowsets_to_clone) { + new_tablet_meta->add_rs_meta(rs_meta); } - LOG(INFO) << "delete version from new local tablet_meta when clone. [table=" - << full_name() << ", version=" << version << "]"; + VLOG_NOTICE << "load rowsets successfully when clone. tablet=" << full_name() + << ", added rowset size=" << rowsets_to_clone.size(); + // save and reload tablet_meta + res = new_tablet_meta->save_meta(_data_dir); + if (!res.ok()) { + LOG(WARNING) << "failed to save new local tablet_meta when clone. res:" << res; + break; + } + _tablet_meta = new_tablet_meta; + } while (0); + + for (auto& version : versions_to_delete) { + auto it = _rs_version_map.find(version); + DCHECK(it != _rs_version_map.end()); + StorageEngine::instance()->add_unused_rowset(it->second); + _rs_version_map.erase(it); } - // add new cloned rowset for (auto& rs_meta : rowsets_to_clone) { - new_tablet_meta->add_rs_meta(rs_meta); + Version version = {rs_meta->start_version(), rs_meta->end_version()}; + RowsetSharedPtr rowset; + res = RowsetFactory::create_rowset(&_schema, _tablet_path, rs_meta, &rowset); + if (!res.ok()) { + LOG(WARNING) << "fail to init rowset. version=" << version; + return res; + } + _rs_version_map[version] = std::move(rowset); } - VLOG_NOTICE << "load rowsets successfully when clone. tablet=" << full_name() - << ", added rowset size=" << rowsets_to_clone.size(); - // save and reload tablet_meta - res = new_tablet_meta->save_meta(_data_dir); - if (!res.ok()) { - LOG(WARNING) << "failed to save new local tablet_meta when clone. res:" << res; - break; + + // reconstruct from tablet meta + _timestamped_version_tracker.construct_versioned_tracker(_tablet_meta->all_rs_metas()); + // clear stale rowset + for (auto& it : _stale_rs_version_map) { + StorageEngine::instance()->add_unused_rowset(it.second); } - _tablet_meta = new_tablet_meta; - } while (0); + _stale_rs_version_map.clear(); + _tablet_meta->clear_stale_rowset(); - for (auto& version : versions_to_delete) { - auto it = _rs_version_map.find(version); - DCHECK(it != _rs_version_map.end()); - StorageEngine::instance()->add_unused_rowset(it->second); - _rs_version_map.erase(it); + LOG(INFO) << "finish to revise tablet. res=" << res << ", " + << "table=" << full_name(); + return res; } - for (auto& rs_meta : rowsets_to_clone) { - Version version = {rs_meta->start_version(), rs_meta->end_version()}; - RowsetSharedPtr rowset; - res = RowsetFactory::create_rowset(&_schema, _tablet_path, rs_meta, &rowset); - if (!res.ok()) { - LOG(WARNING) << "fail to init rowset. version=" << version; - return res; + Status Tablet::add_rowset(RowsetSharedPtr rowset) { + DCHECK(rowset != nullptr); + std::lock_guard wrlock(_meta_lock); + // If the rowset already exist, just return directly. The rowset_id is an unique-id, + // we can use it to check this situation. + if (_contains_rowset(rowset->rowset_id())) { + return Status::OK(); } - _rs_version_map[version] = std::move(rowset); - } - - // reconstruct from tablet meta - _timestamped_version_tracker.construct_versioned_tracker(_tablet_meta->all_rs_metas()); - // clear stale rowset - for (auto& it : _stale_rs_version_map) { - StorageEngine::instance()->add_unused_rowset(it.second); - } - _stale_rs_version_map.clear(); - _tablet_meta->clear_stale_rowset(); + // Otherwise, the version should be not contained in any existing rowset. + RETURN_NOT_OK(_contains_version(rowset->version())); - LOG(INFO) << "finish to revise tablet. res=" << res << ", " - << "table=" << full_name(); - return res; -} + RETURN_NOT_OK(_tablet_meta->add_rs_meta(rowset->rowset_meta())); + _rs_version_map[rowset->version()] = rowset; + _timestamped_version_tracker.add_version(rowset->version()); -Status Tablet::add_rowset(RowsetSharedPtr rowset) { - DCHECK(rowset != nullptr); - std::lock_guard wrlock(_meta_lock); - // If the rowset already exist, just return directly. The rowset_id is an unique-id, - // we can use it to check this situation. - if (_contains_rowset(rowset->rowset_id())) { + std::vector rowsets_to_delete; + // yiguolei: temp code, should remove the rowset contains by this rowset + // but it should be removed in multi path version + for (auto& it : _rs_version_map) { + if (rowset->version().contains(it.first) && rowset->version() != it.first) { + CHECK(it.second != nullptr) + << "there exist a version=" << it.first + << " contains the input rs with version=" << rowset->version() + << ", but the related rs is null"; + rowsets_to_delete.push_back(it.second); + } + } + std::vector empty_vec; + modify_rowsets(empty_vec, rowsets_to_delete); + ++_newly_created_rowset_num; return Status::OK(); } - // Otherwise, the version should be not contained in any existing rowset. - RETURN_NOT_OK(_contains_version(rowset->version())); - - RETURN_NOT_OK(_tablet_meta->add_rs_meta(rowset->rowset_meta())); - _rs_version_map[rowset->version()] = rowset; - _timestamped_version_tracker.add_version(rowset->version()); - - std::vector rowsets_to_delete; - // yiguolei: temp code, should remove the rowset contains by this rowset - // but it should be removed in multi path version - for (auto& it : _rs_version_map) { - if (rowset->version().contains(it.first) && rowset->version() != it.first) { - CHECK(it.second != nullptr) - << "there exist a version=" << it.first - << " contains the input rs with version=" << rowset->version() - << ", but the related rs is null"; - rowsets_to_delete.push_back(it.second); - } - } - std::vector empty_vec; - modify_rowsets(empty_vec, rowsets_to_delete); - ++_newly_created_rowset_num; - return Status::OK(); -} - -Status Tablet::modify_rowsets(std::vector& to_add, - std::vector& to_delete, bool check_delete) { - // the compaction process allow to compact the single version, eg: version[4-4]. - // this kind of "single version compaction" has same "input version" and "output version". - // which means "to_add->version()" equals to "to_delete->version()". - // So we should delete the "to_delete" before adding the "to_add", - // otherwise, the "to_add" will be deleted from _rs_version_map, eventually. - // - // And if the version of "to_add" and "to_delete" are exactly same. eg: - // to_add: [7-7] - // to_delete: [7-7] - // In this case, we no longer need to add the rowset in "to_delete" to - // _stale_rs_version_map, but can delete it directly. - - bool same_version = true; - std::sort(to_add.begin(), to_add.end(), Rowset::comparator); - std::sort(to_delete.begin(), to_delete.end(), Rowset::comparator); - if (to_add.size() == to_delete.size()) { - for (int i = 0; i < to_add.size(); ++i) { - if (to_add[i]->version() != to_delete[i]->version()) { - same_version = false; - break; + + Status Tablet::modify_rowsets(std::vector & to_add, + std::vector & to_delete, bool check_delete) { + // the compaction process allow to compact the single version, eg: version[4-4]. + // this kind of "single version compaction" has same "input version" and "output version". + // which means "to_add->version()" equals to "to_delete->version()". + // So we should delete the "to_delete" before adding the "to_add", + // otherwise, the "to_add" will be deleted from _rs_version_map, eventually. + // + // And if the version of "to_add" and "to_delete" are exactly same. eg: + // to_add: [7-7] + // to_delete: [7-7] + // In this case, we no longer need to add the rowset in "to_delete" to + // _stale_rs_version_map, but can delete it directly. + + bool same_version = true; + std::sort(to_add.begin(), to_add.end(), Rowset::comparator); + std::sort(to_delete.begin(), to_delete.end(), Rowset::comparator); + if (to_add.size() == to_delete.size()) { + for (int i = 0; i < to_add.size(); ++i) { + if (to_add[i]->version() != to_delete[i]->version()) { + same_version = false; + break; + } + } + } else { + same_version = false; + } + + if (check_delete) { + for (auto& rs : to_delete) { + auto find_rs = _rs_version_map.find(rs->version()); + if (find_rs == _rs_version_map.end()) { + LOG(WARNING) << "try to delete not exist version " << rs->version() << " from " + << full_name(); + return Status::OLAPInternalError(OLAP_ERR_DELETE_VERSION_ERROR); + } else if (find_rs->second->rowset_id() != rs->rowset_id()) { + LOG(WARNING) << "try to delete version " << rs->version() << " from " + << full_name() << ", but rowset id changed, delete rowset id is " + << rs->rowset_id() << ", exists rowsetid is" + << find_rs->second->rowset_id(); + return Status::OLAPInternalError(OLAP_ERR_DELETE_VERSION_ERROR); + } } } - } else { - same_version = false; - } - if (check_delete) { + std::vector rs_metas_to_delete; for (auto& rs : to_delete) { - auto find_rs = _rs_version_map.find(rs->version()); - if (find_rs == _rs_version_map.end()) { - LOG(WARNING) << "try to delete not exist version " << rs->version() << " from " - << full_name(); - return Status::OLAPInternalError(OLAP_ERR_DELETE_VERSION_ERROR); - } else if (find_rs->second->rowset_id() != rs->rowset_id()) { - LOG(WARNING) << "try to delete version " << rs->version() << " from " << full_name() - << ", but rowset id changed, delete rowset id is " << rs->rowset_id() - << ", exists rowsetid is" << find_rs->second->rowset_id(); - return Status::OLAPInternalError(OLAP_ERR_DELETE_VERSION_ERROR); + rs_metas_to_delete.push_back(rs->rowset_meta()); + _rs_version_map.erase(rs->version()); + + if (!same_version) { + // put compaction rowsets in _stale_rs_version_map. + _stale_rs_version_map[rs->version()] = rs; } } - } - std::vector rs_metas_to_delete; - for (auto& rs : to_delete) { - rs_metas_to_delete.push_back(rs->rowset_meta()); - _rs_version_map.erase(rs->version()); + std::vector rs_metas_to_add; + for (auto& rs : to_add) { + rs_metas_to_add.push_back(rs->rowset_meta()); + _rs_version_map[rs->version()] = rs; - if (!same_version) { - // put compaction rowsets in _stale_rs_version_map. - _stale_rs_version_map[rs->version()] = rs; + if (!same_version) { + // If version are same, then _timestamped_version_tracker + // already has this version, no need to add again. + _timestamped_version_tracker.add_version(rs->version()); + } + ++_newly_created_rowset_num; } - } - std::vector rs_metas_to_add; - for (auto& rs : to_add) { - rs_metas_to_add.push_back(rs->rowset_meta()); - _rs_version_map[rs->version()] = rs; + _tablet_meta->modify_rs_metas(rs_metas_to_add, rs_metas_to_delete, same_version); if (!same_version) { - // If version are same, then _timestamped_version_tracker - // already has this version, no need to add again. - _timestamped_version_tracker.add_version(rs->version()); + // add rs_metas_to_delete to tracker + _timestamped_version_tracker.add_stale_path_version(rs_metas_to_delete); + } else { + // delete rowset in "to_delete" directly + for (auto& rs : to_delete) { + LOG(INFO) << "add unused rowset " << rs->rowset_id() << " because of same version"; + StorageEngine::instance()->add_unused_rowset(rs); + } } - ++_newly_created_rowset_num; + return Status::OK(); } - _tablet_meta->modify_rs_metas(rs_metas_to_add, rs_metas_to_delete, same_version); - - if (!same_version) { - // add rs_metas_to_delete to tracker - _timestamped_version_tracker.add_stale_path_version(rs_metas_to_delete); - } else { - // delete rowset in "to_delete" directly - for (auto& rs : to_delete) { - LOG(INFO) << "add unused rowset " << rs->rowset_id() << " because of same version"; - StorageEngine::instance()->add_unused_rowset(rs); + // snapshot manager may call this api to check if version exists, so that + // the version maybe not exist + const RowsetSharedPtr Tablet::get_rowset_by_version(const Version& version, bool find_in_stale) + const { + auto iter = _rs_version_map.find(version); + if (iter == _rs_version_map.end()) { + if (find_in_stale) { + return get_stale_rowset_by_version(version); + } + return nullptr; } + return iter->second; } - return Status::OK(); -} -// snapshot manager may call this api to check if version exists, so that -// the version maybe not exist -const RowsetSharedPtr Tablet::get_rowset_by_version(const Version& version, - bool find_in_stale) const { - auto iter = _rs_version_map.find(version); - if (iter == _rs_version_map.end()) { - if (find_in_stale) { - return get_stale_rowset_by_version(version); + const RowsetSharedPtr Tablet::get_stale_rowset_by_version(const Version& version) const { + auto iter = _stale_rs_version_map.find(version); + if (iter == _stale_rs_version_map.end()) { + VLOG_NOTICE << "no rowset for version:" << version << ", tablet: " << full_name(); + return nullptr; } - return nullptr; + return iter->second; } - return iter->second; -} -const RowsetSharedPtr Tablet::get_stale_rowset_by_version(const Version& version) const { - auto iter = _stale_rs_version_map.find(version); - if (iter == _stale_rs_version_map.end()) { - VLOG_NOTICE << "no rowset for version:" << version << ", tablet: " << full_name(); - return nullptr; - } - return iter->second; -} + // Already under _meta_lock + const RowsetSharedPtr Tablet::rowset_with_max_version() const { + Version max_version = _tablet_meta->max_version(); + if (max_version.first == -1) { + return nullptr; + } -// Already under _meta_lock -const RowsetSharedPtr Tablet::rowset_with_max_version() const { - Version max_version = _tablet_meta->max_version(); - if (max_version.first == -1) { - return nullptr; + auto iter = _rs_version_map.find(max_version); + if (iter == _rs_version_map.end()) { + DCHECK(false) << "invalid version:" << max_version; + return nullptr; + } + return iter->second; } - auto iter = _rs_version_map.find(max_version); - if (iter == _rs_version_map.end()) { - DCHECK(false) << "invalid version:" << max_version; - return nullptr; + const RowsetMetaSharedPtr Tablet::rowset_meta_with_max_schema_version( + const std::vector& rowset_metas) { + return *std::max_element(rowset_metas.begin(), rowset_metas.end(), + [](const RowsetMetaSharedPtr& a, const RowsetMetaSharedPtr& b) { + return a->tablet_schema()->schema_version() < + b->tablet_schema()->schema_version(); + }); } - return iter->second; -} -RowsetSharedPtr Tablet::_rowset_with_largest_size() { - RowsetSharedPtr largest_rowset = nullptr; - for (auto& it : _rs_version_map) { - if (it.second->empty() || it.second->zero_num_rows()) { - continue; - } - if (largest_rowset == nullptr || it.second->rowset_meta()->index_disk_size() > - largest_rowset->rowset_meta()->index_disk_size()) { - largest_rowset = it.second; + RowsetSharedPtr Tablet::_rowset_with_largest_size() { + RowsetSharedPtr largest_rowset = nullptr; + for (auto& it : _rs_version_map) { + if (it.second->empty() || it.second->zero_num_rows()) { + continue; + } + if (largest_rowset == nullptr || + it.second->rowset_meta()->index_disk_size() > + largest_rowset->rowset_meta()->index_disk_size()) { + largest_rowset = it.second; + } } - } - - return largest_rowset; -} -// add inc rowset should not persist tablet meta, because it will be persisted when publish txn. -Status Tablet::add_inc_rowset(const RowsetSharedPtr& rowset) { - DCHECK(rowset != nullptr); - std::lock_guard wrlock(_meta_lock); - if (_contains_rowset(rowset->rowset_id())) { - return Status::OK(); + return largest_rowset; } - RETURN_NOT_OK(_contains_version(rowset->version())); - - RETURN_NOT_OK(_tablet_meta->add_rs_meta(rowset->rowset_meta())); - _rs_version_map[rowset->version()] = rowset; - - _timestamped_version_tracker.add_version(rowset->version()); - ++_newly_created_rowset_num; - return Status::OK(); -} - -void Tablet::_delete_stale_rowset_by_version(const Version& version) { - RowsetMetaSharedPtr rowset_meta = _tablet_meta->acquire_stale_rs_meta_by_version(version); - if (rowset_meta == nullptr) { - return; - } - _tablet_meta->delete_stale_rs_meta_by_version(version); - VLOG_NOTICE << "delete stale rowset. tablet=" << full_name() << ", version=" << version; -} + // add inc rowset should not persist tablet meta, because it will be persisted when publish txn. + Status Tablet::add_inc_rowset(const RowsetSharedPtr& rowset) { + DCHECK(rowset != nullptr); + std::lock_guard wrlock(_meta_lock); + if (_contains_rowset(rowset->rowset_id())) { + return Status::OK(); + } + RETURN_NOT_OK(_contains_version(rowset->version())); -void Tablet::delete_expired_stale_rowset() { - int64_t now = UnixSeconds(); - std::lock_guard wrlock(_meta_lock); - // Compute the end time to delete rowsets, when a expired rowset createtime less then this time, it will be deleted. - double expired_stale_sweep_endtime = - ::difftime(now, config::tablet_rowset_stale_sweep_time_sec); + RETURN_NOT_OK(_tablet_meta->add_rs_meta(rowset->rowset_meta())); + _rs_version_map[rowset->version()] = rowset; - std::vector path_id_vec; - // capture the path version to delete - _timestamped_version_tracker.capture_expired_paths( - static_cast(expired_stale_sweep_endtime), &path_id_vec); + _timestamped_version_tracker.add_version(rowset->version()); - if (path_id_vec.empty()) { - return; + ++_newly_created_rowset_num; + return Status::OK(); } - const RowsetSharedPtr lastest_delta = rowset_with_max_version(); - if (lastest_delta == nullptr) { - LOG(WARNING) << "lastest_delta is null " << tablet_id(); - return; + void Tablet::_delete_stale_rowset_by_version(const Version& version) { + RowsetMetaSharedPtr rowset_meta = _tablet_meta->acquire_stale_rs_meta_by_version(version); + if (rowset_meta == nullptr) { + return; + } + _tablet_meta->delete_stale_rs_meta_by_version(version); + VLOG_NOTICE << "delete stale rowset. tablet=" << full_name() << ", version=" << version; } - // fetch missing version before delete - std::vector missed_versions; - calc_missed_versions_unlocked(lastest_delta->end_version(), &missed_versions); - - if (!missed_versions.empty()) { - LOG(WARNING) << "tablet:" << full_name() - << ", missed version for version:" << lastest_delta->end_version(); - _print_missed_versions(missed_versions); - return; - } + void Tablet::delete_expired_stale_rowset() { + int64_t now = UnixSeconds(); + std::lock_guard wrlock(_meta_lock); + // Compute the end time to delete rowsets, when a expired rowset createtime less then this time, it will be deleted. + double expired_stale_sweep_endtime = + ::difftime(now, config::tablet_rowset_stale_sweep_time_sec); - // do check consistent operation - auto path_id_iter = path_id_vec.begin(); + std::vector path_id_vec; + // capture the path version to delete + _timestamped_version_tracker.capture_expired_paths( + static_cast(expired_stale_sweep_endtime), &path_id_vec); - std::map stale_version_path_map; - while (path_id_iter != path_id_vec.end()) { - PathVersionListSharedPtr version_path = - _timestamped_version_tracker.fetch_and_delete_path_by_id(*path_id_iter); + if (path_id_vec.empty()) { + return; + } - Version test_version = Version(0, lastest_delta->end_version()); - stale_version_path_map[*path_id_iter] = version_path; + const RowsetSharedPtr lastest_delta = rowset_with_max_version(); + if (lastest_delta == nullptr) { + LOG(WARNING) << "lastest_delta is null " << tablet_id(); + return; + } - Status status = capture_consistent_versions(test_version, nullptr); - // 1. When there is no consistent versions, we must reconstruct the tracker. - if (!status.ok()) { - // 2. fetch missing version after delete - std::vector after_missed_versions; - calc_missed_versions_unlocked(lastest_delta->end_version(), &after_missed_versions); + // fetch missing version before delete + std::vector missed_versions; + calc_missed_versions_unlocked(lastest_delta->end_version(), &missed_versions); - // 2.1 check whether missed_versions and after_missed_versions are the same. - // when they are the same, it means we can delete the path securely. - bool is_missing = missed_versions.size() != after_missed_versions.size(); + if (!missed_versions.empty()) { + LOG(WARNING) << "tablet:" << full_name() + << ", missed version for version:" << lastest_delta->end_version(); + _print_missed_versions(missed_versions); + return; + } - if (!is_missing) { - for (int ver_index = 0; ver_index < missed_versions.size(); ver_index++) { - if (missed_versions[ver_index] != after_missed_versions[ver_index]) { - is_missing = true; - break; - } - } - } + // do check consistent operation + auto path_id_iter = path_id_vec.begin(); - if (is_missing) { - LOG(WARNING) << "The consistent version check fails, there are bugs. " - << "Reconstruct the tracker to recover versions in tablet=" - << tablet_id(); + std::map stale_version_path_map; + while (path_id_iter != path_id_vec.end()) { + PathVersionListSharedPtr version_path = + _timestamped_version_tracker.fetch_and_delete_path_by_id(*path_id_iter); - // 3. try to recover - _timestamped_version_tracker.recover_versioned_tracker(stale_version_path_map); + Version test_version = Version(0, lastest_delta->end_version()); + stale_version_path_map[*path_id_iter] = version_path; - // 4. double check the consistent versions - // fetch missing version after recover - std::vector recover_missed_versions; - calc_missed_versions_unlocked(lastest_delta->end_version(), - &recover_missed_versions); + Status status = capture_consistent_versions(test_version, nullptr); + // 1. When there is no consistent versions, we must reconstruct the tracker. + if (!status.ok()) { + // 2. fetch missing version after delete + std::vector after_missed_versions; + calc_missed_versions_unlocked(lastest_delta->end_version(), &after_missed_versions); - // 4.1 check whether missed_versions and recover_missed_versions are the same. - // when they are the same, it means we recover successfully. - bool is_recover_missing = missed_versions.size() != recover_missed_versions.size(); + // 2.1 check whether missed_versions and after_missed_versions are the same. + // when they are the same, it means we can delete the path securely. + bool is_missing = missed_versions.size() != after_missed_versions.size(); - if (!is_recover_missing) { + if (!is_missing) { for (int ver_index = 0; ver_index < missed_versions.size(); ver_index++) { - if (missed_versions[ver_index] != recover_missed_versions[ver_index]) { - is_recover_missing = true; + if (missed_versions[ver_index] != after_missed_versions[ver_index]) { + is_missing = true; break; } } } - // 5. check recover fail, version is mission - if (is_recover_missing) { - if (!config::ignore_rowset_stale_unconsistent_delete) { - LOG(FATAL) << "rowset stale unconsistent delete. tablet= " << tablet_id(); - } else { - LOG(WARNING) << "rowset stale unconsistent delete. tablet= " << tablet_id(); + if (is_missing) { + LOG(WARNING) << "The consistent version check fails, there are bugs. " + << "Reconstruct the tracker to recover versions in tablet=" + << tablet_id(); + + // 3. try to recover + _timestamped_version_tracker.recover_versioned_tracker(stale_version_path_map); + + // 4. double check the consistent versions + // fetch missing version after recover + std::vector recover_missed_versions; + calc_missed_versions_unlocked(lastest_delta->end_version(), + &recover_missed_versions); + + // 4.1 check whether missed_versions and recover_missed_versions are the same. + // when they are the same, it means we recover successfully. + bool is_recover_missing = + missed_versions.size() != recover_missed_versions.size(); + + if (!is_recover_missing) { + for (int ver_index = 0; ver_index < missed_versions.size(); ver_index++) { + if (missed_versions[ver_index] != recover_missed_versions[ver_index]) { + is_recover_missing = true; + break; + } + } + } + + // 5. check recover fail, version is mission + if (is_recover_missing) { + if (!config::ignore_rowset_stale_unconsistent_delete) { + LOG(FATAL) + << "rowset stale unconsistent delete. tablet= " << tablet_id(); + } else { + LOG(WARNING) + << "rowset stale unconsistent delete. tablet= " << tablet_id(); + } } } + return; } - return; - } - path_id_iter++; - } - - auto old_size = _stale_rs_version_map.size(); - auto old_meta_size = _tablet_meta->all_stale_rs_metas().size(); - - // do delete operation - auto to_delete_iter = stale_version_path_map.begin(); - while (to_delete_iter != stale_version_path_map.end()) { - std::vector& to_delete_version = - to_delete_iter->second->timestamped_versions(); - for (auto& timestampedVersion : to_delete_version) { - auto it = _stale_rs_version_map.find(timestampedVersion->version()); - if (it != _stale_rs_version_map.end()) { - // delete rowset - StorageEngine::instance()->add_unused_rowset(it->second); - _stale_rs_version_map.erase(it); - VLOG_NOTICE << "delete stale rowset tablet=" << full_name() << " version[" - << timestampedVersion->version().first << "," - << timestampedVersion->version().second - << "] move to unused_rowset success " << std::fixed - << expired_stale_sweep_endtime; - } else { - LOG(WARNING) << "delete stale rowset tablet=" << full_name() << " version[" - << timestampedVersion->version().first << "," - << timestampedVersion->version().second - << "] not find in stale rs version map"; + path_id_iter++; + } + + auto old_size = _stale_rs_version_map.size(); + auto old_meta_size = _tablet_meta->all_stale_rs_metas().size(); + + // do delete operation + auto to_delete_iter = stale_version_path_map.begin(); + while (to_delete_iter != stale_version_path_map.end()) { + std::vector& to_delete_version = + to_delete_iter->second->timestamped_versions(); + for (auto& timestampedVersion : to_delete_version) { + auto it = _stale_rs_version_map.find(timestampedVersion->version()); + if (it != _stale_rs_version_map.end()) { + // delete rowset + StorageEngine::instance()->add_unused_rowset(it->second); + _stale_rs_version_map.erase(it); + VLOG_NOTICE << "delete stale rowset tablet=" << full_name() << " version[" + << timestampedVersion->version().first << "," + << timestampedVersion->version().second + << "] move to unused_rowset success " << std::fixed + << expired_stale_sweep_endtime; + } else { + LOG(WARNING) << "delete stale rowset tablet=" << full_name() << " version[" + << timestampedVersion->version().first << "," + << timestampedVersion->version().second + << "] not find in stale rs version map"; + } + _delete_stale_rowset_by_version(timestampedVersion->version()); } - _delete_stale_rowset_by_version(timestampedVersion->version()); + to_delete_iter++; } - to_delete_iter++; - } - bool reconstructed = _reconstruct_version_tracker_if_necessary(); + bool reconstructed = _reconstruct_version_tracker_if_necessary(); - VLOG_NOTICE << "delete stale rowset _stale_rs_version_map tablet=" << full_name() - << " current_size=" << _stale_rs_version_map.size() << " old_size=" << old_size - << " current_meta_size=" << _tablet_meta->all_stale_rs_metas().size() - << " old_meta_size=" << old_meta_size << " sweep endtime " << std::fixed - << expired_stale_sweep_endtime << ", reconstructed=" << reconstructed; + VLOG_NOTICE << "delete stale rowset _stale_rs_version_map tablet=" << full_name() + << " current_size=" << _stale_rs_version_map.size() << " old_size=" << old_size + << " current_meta_size=" << _tablet_meta->all_stale_rs_metas().size() + << " old_meta_size=" << old_meta_size << " sweep endtime " << std::fixed + << expired_stale_sweep_endtime << ", reconstructed=" << reconstructed; #ifndef BE_TEST - save_meta(); + save_meta(); #endif -} + } -bool Tablet::_reconstruct_version_tracker_if_necessary() { - double orphan_vertex_ratio = _timestamped_version_tracker.get_orphan_vertex_ratio(); - if (orphan_vertex_ratio >= config::tablet_version_graph_orphan_vertex_ratio) { - _timestamped_version_tracker.construct_versioned_tracker( - _tablet_meta->all_rs_metas(), _tablet_meta->all_stale_rs_metas()); - return true; + bool Tablet::_reconstruct_version_tracker_if_necessary() { + double orphan_vertex_ratio = _timestamped_version_tracker.get_orphan_vertex_ratio(); + if (orphan_vertex_ratio >= config::tablet_version_graph_orphan_vertex_ratio) { + _timestamped_version_tracker.construct_versioned_tracker( + _tablet_meta->all_rs_metas(), _tablet_meta->all_stale_rs_metas()); + return true; + } + return false; } - return false; -} -Status Tablet::capture_consistent_versions(const Version& spec_version, - std::vector* version_path, bool quiet) const { - Status status = - _timestamped_version_tracker.capture_consistent_versions(spec_version, version_path); - if (!status.ok() && !quiet) { - std::vector missed_versions; - calc_missed_versions_unlocked(spec_version.second, &missed_versions); - if (missed_versions.empty()) { - // if version_path is null, it may be a compaction check logic. - // so to avoid print too many logs. - if (version_path != nullptr) { - LOG(WARNING) << "tablet:" << full_name() - << ", version already has been merged. spec_version: " << spec_version; - } - status = Status::OLAPInternalError(OLAP_ERR_VERSION_ALREADY_MERGED); - } else { - if (version_path != nullptr) { - LOG(WARNING) << "status:" << status << ", tablet:" << full_name() - << ", missed version for version:" << spec_version; - _print_missed_versions(missed_versions); + Status Tablet::capture_consistent_versions( + const Version& spec_version, std::vector* version_path, bool quiet) const { + Status status = _timestamped_version_tracker.capture_consistent_versions(spec_version, + version_path); + if (!status.ok() && !quiet) { + std::vector missed_versions; + calc_missed_versions_unlocked(spec_version.second, &missed_versions); + if (missed_versions.empty()) { + // if version_path is null, it may be a compaction check logic. + // so to avoid print too many logs. + if (version_path != nullptr) { + LOG(WARNING) << "tablet:" << full_name() + << ", version already has been merged. spec_version: " + << spec_version; + } + status = Status::OLAPInternalError(OLAP_ERR_VERSION_ALREADY_MERGED); + } else { + if (version_path != nullptr) { + LOG(WARNING) << "status:" << status << ", tablet:" << full_name() + << ", missed version for version:" << spec_version; + _print_missed_versions(missed_versions); + } } } + return status; } - return status; -} -Status Tablet::check_version_integrity(const Version& version, bool quiet) { - std::shared_lock rdlock(_meta_lock); - return capture_consistent_versions(version, nullptr, quiet); -} + Status Tablet::check_version_integrity(const Version& version, bool quiet) { + std::shared_lock rdlock(_meta_lock); + return capture_consistent_versions(version, nullptr, quiet); + } -// If any rowset contains the specific version, it means the version already exist -bool Tablet::check_version_exist(const Version& version) const { - for (auto& it : _rs_version_map) { - if (it.first.contains(version)) { - return true; + // If any rowset contains the specific version, it means the version already exist + bool Tablet::check_version_exist(const Version& version) const { + for (auto& it : _rs_version_map) { + if (it.first.contains(version)) { + return true; + } } + return false; } - return false; -} -// The meta read lock should be held before calling -void Tablet::acquire_version_and_rowsets( - std::vector>* version_rowsets) const { - for (const auto& it : _rs_version_map) { - version_rowsets->emplace_back(it.first, it.second); + // The meta read lock should be held before calling + void Tablet::acquire_version_and_rowsets(std::vector> * + version_rowsets) const { + for (const auto& it : _rs_version_map) { + version_rowsets->emplace_back(it.first, it.second); + } } -} -Status Tablet::capture_consistent_rowsets(const Version& spec_version, - std::vector* rowsets) const { - std::vector version_path; - RETURN_NOT_OK(capture_consistent_versions(spec_version, &version_path)); - RETURN_NOT_OK(_capture_consistent_rowsets_unlocked(version_path, rowsets)); - return Status::OK(); -} + Status Tablet::capture_consistent_rowsets(const Version& spec_version, + std::vector* rowsets) const { + std::vector version_path; + RETURN_NOT_OK(capture_consistent_versions(spec_version, &version_path)); + RETURN_NOT_OK(_capture_consistent_rowsets_unlocked(version_path, rowsets)); + return Status::OK(); + } -Status Tablet::_capture_consistent_rowsets_unlocked(const std::vector& version_path, - std::vector* rowsets) const { - DCHECK(rowsets != nullptr && rowsets->empty()); - rowsets->reserve(version_path.size()); - for (auto& version : version_path) { - bool is_find = false; - do { - auto it = _rs_version_map.find(version); - if (it != _rs_version_map.end()) { - is_find = true; - rowsets->push_back(it->second); - break; - } + Status Tablet::_capture_consistent_rowsets_unlocked( + const std::vector& version_path, std::vector* rowsets) const { + DCHECK(rowsets != nullptr && rowsets->empty()); + rowsets->reserve(version_path.size()); + for (auto& version : version_path) { + bool is_find = false; + do { + auto it = _rs_version_map.find(version); + if (it != _rs_version_map.end()) { + is_find = true; + rowsets->push_back(it->second); + break; + } - auto it_expired = _stale_rs_version_map.find(version); - if (it_expired != _stale_rs_version_map.end()) { - is_find = true; - rowsets->push_back(it_expired->second); - break; - } - } while (0); + auto it_expired = _stale_rs_version_map.find(version); + if (it_expired != _stale_rs_version_map.end()) { + is_find = true; + rowsets->push_back(it_expired->second); + break; + } + } while (0); - if (!is_find) { - LOG(WARNING) << "fail to find Rowset for version. tablet=" << full_name() - << ", version='" << version; - return Status::OLAPInternalError(OLAP_ERR_CAPTURE_ROWSET_ERROR); - } - } - return Status::OK(); -} - -Status Tablet::capture_rs_readers(const Version& spec_version, - std::vector* rs_readers) const { - std::vector version_path; - RETURN_NOT_OK(capture_consistent_versions(spec_version, &version_path)); - RETURN_NOT_OK(capture_rs_readers(version_path, rs_readers)); - return Status::OK(); -} - -Status Tablet::capture_rs_readers(const std::vector& version_path, - std::vector* rs_readers) const { - DCHECK(rs_readers != nullptr && rs_readers->empty()); - for (auto version : version_path) { - auto it = _rs_version_map.find(version); - if (it == _rs_version_map.end()) { - VLOG_NOTICE << "fail to find Rowset in rs_version for version. tablet=" << full_name() - << ", version='" << version.first << "-" << version.second; - - it = _stale_rs_version_map.find(version); - if (it == _stale_rs_version_map.end()) { - LOG(WARNING) << "fail to find Rowset in stale_rs_version for version. tablet=" - << full_name() << ", version='" << version.first << "-" - << version.second; - return Status::OLAPInternalError(OLAP_ERR_CAPTURE_ROWSET_READER_ERROR); + if (!is_find) { + LOG(WARNING) << "fail to find Rowset for version. tablet=" << full_name() + << ", version='" << version; + return Status::OLAPInternalError(OLAP_ERR_CAPTURE_ROWSET_ERROR); } } - RowsetReaderSharedPtr rs_reader; - auto res = it->second->create_reader(&rs_reader); - if (!res.ok()) { - LOG(WARNING) << "failed to create reader for rowset:" << it->second->rowset_id(); - return Status::OLAPInternalError(OLAP_ERR_CAPTURE_ROWSET_READER_ERROR); - } - rs_readers->push_back(std::move(rs_reader)); + return Status::OK(); } - return Status::OK(); -} - -void Tablet::add_delete_predicate(const DeletePredicatePB& delete_predicate, int64_t version) { - _tablet_meta->add_delete_predicate(delete_predicate, version); -} -// TODO(lingbin): what is the difference between version_for_delete_predicate() and -// version_for_load_deletion()? should at least leave a comment -bool Tablet::version_for_delete_predicate(const Version& version) { - return _tablet_meta->version_for_delete_predicate(version); -} + Status Tablet::capture_rs_readers(const Version& spec_version, + std::vector* rs_readers) const { + std::vector version_path; + RETURN_NOT_OK(capture_consistent_versions(spec_version, &version_path)); + RETURN_NOT_OK(capture_rs_readers(version_path, rs_readers)); + return Status::OK(); + } -bool Tablet::version_for_load_deletion(const Version& version) { - RowsetSharedPtr rowset = _rs_version_map.at(version); - return rowset->delete_flag(); -} + Status Tablet::capture_rs_readers(const std::vector& version_path, + std::vector* rs_readers) const { + DCHECK(rs_readers != nullptr && rs_readers->empty()); + for (auto version : version_path) { + auto it = _rs_version_map.find(version); + if (it == _rs_version_map.end()) { + VLOG_NOTICE << "fail to find Rowset in rs_version for version. tablet=" + << full_name() << ", version='" << version.first << "-" + << version.second; + + it = _stale_rs_version_map.find(version); + if (it == _stale_rs_version_map.end()) { + LOG(WARNING) << "fail to find Rowset in stale_rs_version for version. tablet=" + << full_name() << ", version='" << version.first << "-" + << version.second; + return Status::OLAPInternalError(OLAP_ERR_CAPTURE_ROWSET_READER_ERROR); + } + } + RowsetReaderSharedPtr rs_reader; + auto res = it->second->create_reader(&rs_reader); + if (!res.ok()) { + LOG(WARNING) << "failed to create reader for rowset:" << it->second->rowset_id(); + return Status::OLAPInternalError(OLAP_ERR_CAPTURE_ROWSET_READER_ERROR); + } + rs_readers->push_back(std::move(rs_reader)); + } + return Status::OK(); + } -bool Tablet::can_do_compaction(size_t path_hash, CompactionType compaction_type) { - if (compaction_type == CompactionType::BASE_COMPACTION && tablet_state() != TABLET_RUNNING) { - // base compaction can only be done for tablet in TABLET_RUNNING state. - // but cumulative compaction can be done for TABLET_NOTREADY, such as tablet under alter process. - return false; + void Tablet::add_delete_predicate(const DeletePredicatePB& delete_predicate, int64_t version) { + _tablet_meta->add_delete_predicate(delete_predicate, version); } - if (data_dir()->path_hash() != path_hash || !is_used() || !init_succeeded()) { - return false; + // TODO(lingbin): what is the difference between version_for_delete_predicate() and + // version_for_load_deletion()? should at least leave a comment + bool Tablet::version_for_delete_predicate(const Version& version) { + return _tablet_meta->version_for_delete_predicate(version); } - if (tablet_state() == TABLET_NOTREADY) { - // Before doing schema change, tablet's rowsets that versions smaller than max converting version will be - // removed. So, we only need to do the compaction when it is being converted. - // After being converted, tablet's state will be changed to TABLET_RUNNING. - return SchemaChangeHandler::tablet_in_converting(tablet_id()); + bool Tablet::version_for_load_deletion(const Version& version) { + RowsetSharedPtr rowset = _rs_version_map.at(version); + return rowset->delete_flag(); } - return true; -} + bool Tablet::can_do_compaction(size_t path_hash, CompactionType compaction_type) { + if (compaction_type == CompactionType::BASE_COMPACTION && + tablet_state() != TABLET_RUNNING) { + // base compaction can only be done for tablet in TABLET_RUNNING state. + // but cumulative compaction can be done for TABLET_NOTREADY, such as tablet under alter process. + return false; + } + + if (data_dir()->path_hash() != path_hash || !is_used() || !init_succeeded()) { + return false; + } + + if (tablet_state() == TABLET_NOTREADY) { + // Before doing schema change, tablet's rowsets that versions smaller than max converting version will be + // removed. So, we only need to do the compaction when it is being converted. + // After being converted, tablet's state will be changed to TABLET_RUNNING. + return SchemaChangeHandler::tablet_in_converting(tablet_id()); + } -uint32_t Tablet::calc_compaction_score( - CompactionType compaction_type, - std::shared_ptr cumulative_compaction_policy) { - // Need meta lock, because it will iterator "all_rs_metas" of tablet meta. - std::shared_lock rdlock(_meta_lock); - if (compaction_type == CompactionType::CUMULATIVE_COMPACTION) { - return _calc_cumulative_compaction_score(cumulative_compaction_policy); - } else { - DCHECK_EQ(compaction_type, CompactionType::BASE_COMPACTION); - return _calc_base_compaction_score(); + return true; } -} -const uint32_t Tablet::_calc_cumulative_compaction_score( - std::shared_ptr cumulative_compaction_policy) { -#ifndef BE_TEST - if (_cumulative_compaction_policy == nullptr || - _cumulative_compaction_policy->name() != cumulative_compaction_policy->name()) { - _cumulative_compaction_policy = cumulative_compaction_policy; + uint32_t Tablet::calc_compaction_score( + CompactionType compaction_type, + std::shared_ptr cumulative_compaction_policy) { + // Need meta lock, because it will iterator "all_rs_metas" of tablet meta. + std::shared_lock rdlock(_meta_lock); + if (compaction_type == CompactionType::CUMULATIVE_COMPACTION) { + return _calc_cumulative_compaction_score(cumulative_compaction_policy); + } else { + DCHECK_EQ(compaction_type, CompactionType::BASE_COMPACTION); + return _calc_base_compaction_score(); + } } + + const uint32_t Tablet::_calc_cumulative_compaction_score( + std::shared_ptr cumulative_compaction_policy) { +#ifndef BE_TEST + if (_cumulative_compaction_policy == nullptr || + _cumulative_compaction_policy->name() != cumulative_compaction_policy->name()) { + _cumulative_compaction_policy = cumulative_compaction_policy; + } #endif - uint32_t score = 0; - _cumulative_compaction_policy->calc_cumulative_compaction_score( - tablet_state(), _tablet_meta->all_rs_metas(), cumulative_layer_point(), &score); - return score; -} - -const uint32_t Tablet::_calc_base_compaction_score() const { - uint32_t score = 0; - const int64_t point = cumulative_layer_point(); - bool base_rowset_exist = false; - for (auto& rs_meta : _tablet_meta->all_rs_metas()) { - if (rs_meta->start_version() == 0) { - base_rowset_exist = true; - } - if (rs_meta->start_version() >= point) { - // all_rs_metas() is not sorted, so we use _continue_ other than _break_ here. - continue; - } - - score += rs_meta->get_compaction_score(); - } - - // base不存在可能是tablet正在做alter table,先不选它,设score=0 - return base_rowset_exist ? score : 0; -} - -void Tablet::calc_missed_versions(int64_t spec_version, std::vector* missed_versions) { - std::shared_lock rdlock(_meta_lock); - calc_missed_versions_unlocked(spec_version, missed_versions); -} - -// for example: -// [0-4][5-5][8-8][9-9] -// if spec_version = 6, we still return {7} other than {6, 7} -void Tablet::calc_missed_versions_unlocked(int64_t spec_version, - std::vector* missed_versions) const { - DCHECK(spec_version > 0) << "invalid spec_version: " << spec_version; - std::list existing_versions; - for (auto& rs : _tablet_meta->all_rs_metas()) { - existing_versions.emplace_back(rs->version()); - } - - // sort the existing versions in ascending order - existing_versions.sort([](const Version& a, const Version& b) { - // simple because 2 versions are certainly not overlapping - return a.first < b.first; - }); - - // From the first version(=0), find the missing version until spec_version - int64_t last_version = -1; - for (const Version& version : existing_versions) { - if (version.first > last_version + 1) { - for (int64_t i = last_version + 1; i < version.first && i <= spec_version; ++i) { - missed_versions->emplace_back(Version(i, i)); - } - } - last_version = version.second; - if (last_version >= spec_version) { - break; - } - } - for (int64_t i = last_version + 1; i <= spec_version; ++i) { - missed_versions->emplace_back(Version(i, i)); - } -} - -void Tablet::max_continuous_version_from_beginning(Version* version, Version* max_version) { - bool has_version_cross; - std::shared_lock rdlock(_meta_lock); - _max_continuous_version_from_beginning_unlocked(version, max_version, &has_version_cross); -} - -void Tablet::_max_continuous_version_from_beginning_unlocked(Version* version, Version* max_version, - bool* has_version_cross) const { - std::vector existing_versions; - *has_version_cross = false; - for (auto& rs : _tablet_meta->all_rs_metas()) { - existing_versions.emplace_back(rs->version()); - } - - // sort the existing versions in ascending order - std::sort(existing_versions.begin(), existing_versions.end(), - [](const Version& left, const Version& right) { - // simple because 2 versions are certainly not overlapping - return left.first < right.first; - }); - - Version max_continuous_version = {-1, -1}; - for (int i = 0; i < existing_versions.size(); ++i) { - if (existing_versions[i].first > max_continuous_version.second + 1) { - break; - } else if (existing_versions[i].first <= max_continuous_version.second) { - *has_version_cross = true; - } - max_continuous_version = existing_versions[i]; - } - *version = max_continuous_version; - // tablet may not has rowset, eg, tablet has just been clear for restore. - if (max_version != nullptr && !existing_versions.empty()) { - *max_version = existing_versions.back(); - } -} - -void Tablet::calculate_cumulative_point() { - std::lock_guard wrlock(_meta_lock); - int64_t ret_cumulative_point; - _cumulative_compaction_policy->calculate_cumulative_point( - this, _tablet_meta->all_rs_metas(), _cumulative_point, &ret_cumulative_point); - - if (ret_cumulative_point == K_INVALID_CUMULATIVE_POINT) { - return; + uint32_t score = 0; + _cumulative_compaction_policy->calc_cumulative_compaction_score( + tablet_state(), _tablet_meta->all_rs_metas(), cumulative_layer_point(), &score); + return score; + } + + const uint32_t Tablet::_calc_base_compaction_score() const { + uint32_t score = 0; + const int64_t point = cumulative_layer_point(); + bool base_rowset_exist = false; + for (auto& rs_meta : _tablet_meta->all_rs_metas()) { + if (rs_meta->start_version() == 0) { + base_rowset_exist = true; + } + if (rs_meta->start_version() >= point) { + // all_rs_metas() is not sorted, so we use _continue_ other than _break_ here. + continue; + } + + score += rs_meta->get_compaction_score(); + } + + // base不存在可能是tablet正在做alter table,先不选它,设score=0 + return base_rowset_exist ? score : 0; } - set_cumulative_layer_point(ret_cumulative_point); -} -//find rowsets that rows less then "config::quick_compaction_max_rows" -Status Tablet::pick_quick_compaction_rowsets(std::vector* input_rowsets, - int64_t* permits) { - int max_rows = config::quick_compaction_max_rows; - if (!config::enable_quick_compaction || max_rows <= 0) { - return Status::OK(); + void Tablet::calc_missed_versions(int64_t spec_version, + std::vector * missed_versions) { + std::shared_lock rdlock(_meta_lock); + calc_missed_versions_unlocked(spec_version, missed_versions); } - if (!init_succeeded()) { - return Status::OLAPInternalError(OLAP_ERR_CUMULATIVE_INVALID_PARAMETERS); - } - int max_series_num = 1000; - - std::vector> quick_compaction_rowsets(max_series_num); - int idx = 0; - std::shared_lock rdlock(_meta_lock); - std::vector sortedRowset; - for (auto& rs : _rs_version_map) { - sortedRowset.push_back(rs.second); - } - std::sort(sortedRowset.begin(), sortedRowset.end(), Rowset::comparator); - if (tablet_state() == TABLET_RUNNING) { - for (int i = 0; i < sortedRowset.size(); i++) { - bool is_delete = version_for_delete_predicate(sortedRowset[i]->version()); - if (!is_delete && sortedRowset[i]->start_version() > 0 && - sortedRowset[i]->start_version() > cumulative_layer_point()) { - if (sortedRowset[i]->num_rows() < max_rows) { - quick_compaction_rowsets[idx].push_back(sortedRowset[i]); - } else { - idx++; - if (idx > max_series_num) { - break; - } + + // for example: + // [0-4][5-5][8-8][9-9] + // if spec_version = 6, we still return {7} other than {6, 7} + void Tablet::calc_missed_versions_unlocked(int64_t spec_version, + std::vector * missed_versions) const { + DCHECK(spec_version > 0) << "invalid spec_version: " << spec_version; + std::list existing_versions; + for (auto& rs : _tablet_meta->all_rs_metas()) { + existing_versions.emplace_back(rs->version()); + } + + // sort the existing versions in ascending order + existing_versions.sort([](const Version& a, const Version& b) { + // simple because 2 versions are certainly not overlapping + return a.first < b.first; + }); + + // From the first version(=0), find the missing version until spec_version + int64_t last_version = -1; + for (const Version& version : existing_versions) { + if (version.first > last_version + 1) { + for (int64_t i = last_version + 1; i < version.first && i <= spec_version; ++i) { + missed_versions->emplace_back(Version(i, i)); } } - } - if (quick_compaction_rowsets.size() == 0) return Status::OK(); - std::vector result = quick_compaction_rowsets[0]; - for (int i = 0; i < quick_compaction_rowsets.size(); i++) { - if (quick_compaction_rowsets[i].size() > result.size()) { - result = quick_compaction_rowsets[i]; + last_version = version.second; + if (last_version >= spec_version) { + break; } } - for (int i = 0; i < result.size(); i++) { - *permits += result[i]->num_segments(); - input_rowsets->push_back(result[i]); + for (int64_t i = last_version + 1; i <= spec_version; ++i) { + missed_versions->emplace_back(Version(i, i)); } } - return Status::OK(); -} -Status Tablet::split_range(const OlapTuple& start_key_strings, const OlapTuple& end_key_strings, - uint64_t request_block_row_count, std::vector* ranges) { - DCHECK(ranges != nullptr); + void Tablet::max_continuous_version_from_beginning(Version * version, Version * max_version) { + bool has_version_cross; + std::shared_lock rdlock(_meta_lock); + _max_continuous_version_from_beginning_unlocked(version, max_version, &has_version_cross); + } - size_t key_num = 0; - RowCursor start_key; - // 如果有startkey,用startkey初始化;反之则用minkey初始化 - if (start_key_strings.size() > 0) { - if (start_key.init_scan_key(_schema, start_key_strings.values()) != Status::OK()) { - LOG(WARNING) << "fail to initial key strings with RowCursor type."; - return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED); + void Tablet::_max_continuous_version_from_beginning_unlocked( + Version * version, Version * max_version, bool* has_version_cross) const { + std::vector existing_versions; + *has_version_cross = false; + for (auto& rs : _tablet_meta->all_rs_metas()) { + existing_versions.emplace_back(rs->version()); } - if (start_key.from_tuple(start_key_strings) != Status::OK()) { - LOG(WARNING) << "init end key failed"; - return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA); + // sort the existing versions in ascending order + std::sort(existing_versions.begin(), existing_versions.end(), + [](const Version& left, const Version& right) { + // simple because 2 versions are certainly not overlapping + return left.first < right.first; + }); + + Version max_continuous_version = {-1, -1}; + for (int i = 0; i < existing_versions.size(); ++i) { + if (existing_versions[i].first > max_continuous_version.second + 1) { + break; + } else if (existing_versions[i].first <= max_continuous_version.second) { + *has_version_cross = true; + } + max_continuous_version = existing_versions[i]; } - key_num = start_key_strings.size(); - } else { - if (start_key.init(_schema, num_short_key_columns()) != Status::OK()) { - LOG(WARNING) << "fail to initial key strings with RowCursor type."; - return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED); + *version = max_continuous_version; + // tablet may not has rowset, eg, tablet has just been clear for restore. + if (max_version != nullptr && !existing_versions.empty()) { + *max_version = existing_versions.back(); } - - start_key.allocate_memory_for_string_type(_schema); - start_key.build_min_key(); - key_num = num_short_key_columns(); } - RowCursor end_key; - // 和startkey一样处理,没有则用maxkey初始化 - if (end_key_strings.size() > 0) { - if (!end_key.init_scan_key(_schema, end_key_strings.values())) { - LOG(WARNING) << "fail to parse strings to key with RowCursor type."; - return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA); + void Tablet::calculate_cumulative_point() { + std::lock_guard wrlock(_meta_lock); + int64_t ret_cumulative_point; + _cumulative_compaction_policy->calculate_cumulative_point( + this, _tablet_meta->all_rs_metas(), _cumulative_point, &ret_cumulative_point); + + if (ret_cumulative_point == K_INVALID_CUMULATIVE_POINT) { + return; } + set_cumulative_layer_point(ret_cumulative_point); + } - if (end_key.from_tuple(end_key_strings) != Status::OK()) { - LOG(WARNING) << "init end key failed"; - return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA); + //find rowsets that rows less then "config::quick_compaction_max_rows" + Status Tablet::pick_quick_compaction_rowsets(std::vector * input_rowsets, + int64_t * permits) { + int max_rows = config::quick_compaction_max_rows; + if (!config::enable_quick_compaction || max_rows <= 0) { + return Status::OK(); } - } else { - if (end_key.init(_schema, num_short_key_columns()) != Status::OK()) { - LOG(WARNING) << "fail to initial key strings with RowCursor type."; - return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED); + if (!init_succeeded()) { + return Status::OLAPInternalError(OLAP_ERR_CUMULATIVE_INVALID_PARAMETERS); } + int max_series_num = 1000; - end_key.allocate_memory_for_string_type(_schema); - end_key.build_max_key(); + std::vector> quick_compaction_rowsets(max_series_num); + int idx = 0; + std::shared_lock rdlock(_meta_lock); + std::vector sortedRowset; + for (auto& rs : _rs_version_map) { + sortedRowset.push_back(rs.second); + } + std::sort(sortedRowset.begin(), sortedRowset.end(), Rowset::comparator); + if (tablet_state() == TABLET_RUNNING) { + for (int i = 0; i < sortedRowset.size(); i++) { + bool is_delete = version_for_delete_predicate(sortedRowset[i]->version()); + if (!is_delete && sortedRowset[i]->start_version() > 0 && + sortedRowset[i]->start_version() > cumulative_layer_point()) { + if (sortedRowset[i]->num_rows() < max_rows) { + quick_compaction_rowsets[idx].push_back(sortedRowset[i]); + } else { + idx++; + if (idx > max_series_num) { + break; + } + } + } + } + if (quick_compaction_rowsets.size() == 0) return Status::OK(); + std::vector result = quick_compaction_rowsets[0]; + for (int i = 0; i < quick_compaction_rowsets.size(); i++) { + if (quick_compaction_rowsets[i].size() > result.size()) { + result = quick_compaction_rowsets[i]; + } + } + for (int i = 0; i < result.size(); i++) { + *permits += result[i]->num_segments(); + input_rowsets->push_back(result[i]); + } + } + return Status::OK(); } - std::shared_lock rdlock(_meta_lock); - RowsetSharedPtr rowset = _rowset_with_largest_size(); + Status Tablet::split_range(const OlapTuple& start_key_strings, const OlapTuple& end_key_strings, + uint64_t request_block_row_count, std::vector* ranges) { + DCHECK(ranges != nullptr); - // 如果找不到合适的rowset,就直接返回startkey,endkey - if (rowset == nullptr) { - VLOG_NOTICE << "there is no base file now, may be tablet is empty."; - // it may be right if the tablet is empty, so we return success. - ranges->emplace_back(start_key.to_tuple()); - ranges->emplace_back(end_key.to_tuple()); - return Status::OK(); - } - return rowset->split_range(start_key, end_key, request_block_row_count, key_num, ranges); -} + size_t key_num = 0; + RowCursor start_key; + // 如果有startkey,用startkey初始化;反之则用minkey初始化 + if (start_key_strings.size() > 0) { + if (start_key.init_scan_key(_schema, start_key_strings.values()) != Status::OK()) { + LOG(WARNING) << "fail to initial key strings with RowCursor type."; + return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED); + } -// NOTE: only used when create_table, so it is sure that there is no concurrent reader and writer. -void Tablet::delete_all_files() { - // Release resources like memory and disk space. - std::shared_lock rdlock(_meta_lock); - for (auto it : _rs_version_map) { - it.second->remove(); - } - _rs_version_map.clear(); + if (start_key.from_tuple(start_key_strings) != Status::OK()) { + LOG(WARNING) << "init end key failed"; + return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA); + } + key_num = start_key_strings.size(); + } else { + if (start_key.init(_schema, num_short_key_columns()) != Status::OK()) { + LOG(WARNING) << "fail to initial key strings with RowCursor type."; + return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED); + } - for (auto it : _stale_rs_version_map) { - it.second->remove(); - } - _stale_rs_version_map.clear(); -} + start_key.allocate_memory_for_string_type(_schema); + start_key.build_min_key(); + key_num = num_short_key_columns(); + } -bool Tablet::check_path(const std::string& path_to_check) const { - std::shared_lock rdlock(_meta_lock); - if (path_to_check == _tablet_path) { - return true; + RowCursor end_key; + // 和startkey一样处理,没有则用maxkey初始化 + if (end_key_strings.size() > 0) { + if (!end_key.init_scan_key(_schema, end_key_strings.values())) { + LOG(WARNING) << "fail to parse strings to key with RowCursor type."; + return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA); + } + + if (end_key.from_tuple(end_key_strings) != Status::OK()) { + LOG(WARNING) << "init end key failed"; + return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA); + } + } else { + if (end_key.init(_schema, num_short_key_columns()) != Status::OK()) { + LOG(WARNING) << "fail to initial key strings with RowCursor type."; + return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED); + } + + end_key.allocate_memory_for_string_type(_schema); + end_key.build_max_key(); + } + + std::shared_lock rdlock(_meta_lock); + RowsetSharedPtr rowset = _rowset_with_largest_size(); + + // 如果找不到合适的rowset,就直接返回startkey,endkey + if (rowset == nullptr) { + VLOG_NOTICE << "there is no base file now, may be tablet is empty."; + // it may be right if the tablet is empty, so we return success. + ranges->emplace_back(start_key.to_tuple()); + ranges->emplace_back(end_key.to_tuple()); + return Status::OK(); + } + return rowset->split_range(start_key, end_key, request_block_row_count, key_num, ranges); } - auto tablet_id_dir = io::Path(_tablet_path).parent_path(); - if (path_to_check == tablet_id_dir) { - return true; + + // NOTE: only used when create_table, so it is sure that there is no concurrent reader and writer. + void Tablet::delete_all_files() { + // Release resources like memory and disk space. + std::shared_lock rdlock(_meta_lock); + for (auto it : _rs_version_map) { + it.second->remove(); + } + _rs_version_map.clear(); + + for (auto it : _stale_rs_version_map) { + it.second->remove(); + } + _stale_rs_version_map.clear(); } - for (auto& version_rowset : _rs_version_map) { - bool ret = version_rowset.second->check_path(path_to_check); - if (ret) { + + bool Tablet::check_path(const std::string& path_to_check) const { + std::shared_lock rdlock(_meta_lock); + if (path_to_check == _tablet_path) { return true; } - } - for (auto& stale_version_rowset : _stale_rs_version_map) { - bool ret = stale_version_rowset.second->check_path(path_to_check); - if (ret) { + auto tablet_id_dir = io::Path(_tablet_path).parent_path(); + if (path_to_check == tablet_id_dir) { return true; } + for (auto& version_rowset : _rs_version_map) { + bool ret = version_rowset.second->check_path(path_to_check); + if (ret) { + return true; + } + } + for (auto& stale_version_rowset : _stale_rs_version_map) { + bool ret = stale_version_rowset.second->check_path(path_to_check); + if (ret) { + return true; + } + } + return false; } - return false; -} -// check rowset id in tablet-meta and in rowset-meta atomicly -// for example, during publish version stage, it will first add rowset meta to tablet meta and then -// remove it from rowset meta manager. If we check tablet meta first and then check rowset meta using 2 step unlocked -// the sequence maybe: 1. check in tablet meta [return false] 2. add to tablet meta 3. remove from rowset meta manager -// 4. check in rowset meta manager return false. so that the rowset maybe checked return false it means it is useless and -// will be treated as a garbage. -bool Tablet::check_rowset_id(const RowsetId& rowset_id) { - std::shared_lock rdlock(_meta_lock); - if (StorageEngine::instance()->rowset_id_in_use(rowset_id)) { - return true; - } - for (auto& version_rowset : _rs_version_map) { - if (version_rowset.second->rowset_id() == rowset_id) { + // check rowset id in tablet-meta and in rowset-meta atomicly + // for example, during publish version stage, it will first add rowset meta to tablet meta and then + // remove it from rowset meta manager. If we check tablet meta first and then check rowset meta using 2 step unlocked + // the sequence maybe: 1. check in tablet meta [return false] 2. add to tablet meta 3. remove from rowset meta manager + // 4. check in rowset meta manager return false. so that the rowset maybe checked return false it means it is useless and + // will be treated as a garbage. + bool Tablet::check_rowset_id(const RowsetId& rowset_id) { + std::shared_lock rdlock(_meta_lock); + if (StorageEngine::instance()->rowset_id_in_use(rowset_id)) { return true; } - } - for (auto& stale_version_rowset : _stale_rs_version_map) { - if (stale_version_rowset.second->rowset_id() == rowset_id) { + for (auto& version_rowset : _rs_version_map) { + if (version_rowset.second->rowset_id() == rowset_id) { + return true; + } + } + for (auto& stale_version_rowset : _stale_rs_version_map) { + if (stale_version_rowset.second->rowset_id() == rowset_id) { + return true; + } + } + if (RowsetMetaManager::check_rowset_meta(_data_dir->get_meta(), tablet_uid(), rowset_id)) { return true; } + return false; } - if (RowsetMetaManager::check_rowset_meta(_data_dir->get_meta(), tablet_uid(), rowset_id)) { - return true; + + void Tablet::_print_missed_versions(const std::vector& missed_versions) const { + std::stringstream ss; + ss << full_name() << " has " << missed_versions.size() << " missed version:"; + // print at most 10 version + for (int i = 0; i < 10 && i < missed_versions.size(); ++i) { + ss << missed_versions[i] << ","; + } + LOG(WARNING) << ss.str(); } - return false; -} - -void Tablet::_print_missed_versions(const std::vector& missed_versions) const { - std::stringstream ss; - ss << full_name() << " has " << missed_versions.size() << " missed version:"; - // print at most 10 version - for (int i = 0; i < 10 && i < missed_versions.size(); ++i) { - ss << missed_versions[i] << ","; - } - LOG(WARNING) << ss.str(); -} - -Status Tablet::_contains_version(const Version& version) { - // check if there exist a rowset contains the added rowset - for (auto& it : _rs_version_map) { - if (it.first.contains(version)) { - // TODO(lingbin): Is this check unnecessary? - // because the value type is std::shared_ptr, when will it be nullptr? - // In addition, in this class, there are many places that do not make this judgment - // when access _rs_version_map's value. - CHECK(it.second != nullptr) << "there exist a version=" << it.first - << " contains the input rs with version=" << version - << ", but the related rs is null"; - return Status::OLAPInternalError(OLAP_ERR_PUSH_VERSION_ALREADY_EXIST); - } - } - - return Status::OK(); -} - -Status Tablet::set_partition_id(int64_t partition_id) { - return _tablet_meta->set_partition_id(partition_id); -} - -TabletInfo Tablet::get_tablet_info() const { - return TabletInfo(tablet_id(), schema_hash(), tablet_uid()); -} - -void Tablet::pick_candidate_rowsets_to_cumulative_compaction( - std::vector* candidate_rowsets) { - if (_cumulative_point == K_INVALID_CUMULATIVE_POINT) { - return; - } - std::shared_lock rdlock(_meta_lock); - _cumulative_compaction_policy->pick_candidate_rowsets(_rs_version_map, _cumulative_point, - candidate_rowsets); -} - -void Tablet::pick_candidate_rowsets_to_base_compaction(vector* candidate_rowsets) { - std::shared_lock rdlock(_meta_lock); - // FIXME(cyx): If there are delete predicate rowsets in tablet, - // remote rowsets cannot apply these delete predicate, which can cause - // incorrect query result. - for (auto& it : _rs_version_map) { - if (it.first.first < _cumulative_point && it.second->is_local()) { - candidate_rowsets->push_back(it.second); - } - } -} - -// For http compaction action -void Tablet::get_compaction_status(std::string* json_result) { - rapidjson::Document root; - root.SetObject(); - - rapidjson::Document path_arr; - path_arr.SetArray(); - - std::vector rowsets; - std::vector stale_rowsets; - std::vector delete_flags; - { - std::shared_lock rdlock(_meta_lock); - rowsets.reserve(_rs_version_map.size()); + + Status Tablet::_contains_version(const Version& version) { + // check if there exist a rowset contains the added rowset for (auto& it : _rs_version_map) { - rowsets.push_back(it.second); + if (it.first.contains(version)) { + // TODO(lingbin): Is this check unnecessary? + // because the value type is std::shared_ptr, when will it be nullptr? + // In addition, in this class, there are many places that do not make this judgment + // when access _rs_version_map's value. + CHECK(it.second != nullptr) << "there exist a version=" << it.first + << " contains the input rs with version=" << version + << ", but the related rs is null"; + return Status::OLAPInternalError(OLAP_ERR_PUSH_VERSION_ALREADY_EXIST); + } } - std::sort(rowsets.begin(), rowsets.end(), Rowset::comparator); - stale_rowsets.reserve(_stale_rs_version_map.size()); - for (auto& it : _stale_rs_version_map) { - stale_rowsets.push_back(it.second); - } - std::sort(stale_rowsets.begin(), stale_rowsets.end(), Rowset::comparator); - - delete_flags.reserve(rowsets.size()); - for (auto& rs : rowsets) { - delete_flags.push_back(version_for_delete_predicate(rs->version())); - } - // get snapshot version path json_doc - _timestamped_version_tracker.get_stale_version_path_json_doc(path_arr); - } - rapidjson::Value cumulative_policy_type; - std::string policy_type_str = "cumulative compaction policy not initializied"; - if (_cumulative_compaction_policy != nullptr) { - policy_type_str = _cumulative_compaction_policy->name(); - } - cumulative_policy_type.SetString(policy_type_str.c_str(), policy_type_str.length(), - root.GetAllocator()); - root.AddMember("cumulative policy type", cumulative_policy_type, root.GetAllocator()); - root.AddMember("cumulative point", _cumulative_point.load(), root.GetAllocator()); - rapidjson::Value cumu_value; - std::string format_str = ToStringFromUnixMillis(_last_cumu_compaction_failure_millis.load()); - cumu_value.SetString(format_str.c_str(), format_str.length(), root.GetAllocator()); - root.AddMember("last cumulative failure time", cumu_value, root.GetAllocator()); - rapidjson::Value base_value; - format_str = ToStringFromUnixMillis(_last_base_compaction_failure_millis.load()); - base_value.SetString(format_str.c_str(), format_str.length(), root.GetAllocator()); - root.AddMember("last base failure time", base_value, root.GetAllocator()); - rapidjson::Value cumu_success_value; - format_str = ToStringFromUnixMillis(_last_cumu_compaction_success_millis.load()); - cumu_success_value.SetString(format_str.c_str(), format_str.length(), root.GetAllocator()); - root.AddMember("last cumulative success time", cumu_success_value, root.GetAllocator()); - rapidjson::Value base_success_value; - format_str = ToStringFromUnixMillis(_last_base_compaction_success_millis.load()); - base_success_value.SetString(format_str.c_str(), format_str.length(), root.GetAllocator()); - root.AddMember("last base success time", base_success_value, root.GetAllocator()); - - // print all rowsets' version as an array - rapidjson::Document versions_arr; - rapidjson::Document missing_versions_arr; - versions_arr.SetArray(); - missing_versions_arr.SetArray(); - int64_t last_version = -1; - for (int i = 0; i < rowsets.size(); ++i) { - const Version& ver = rowsets[i]->version(); - if (ver.first != last_version + 1) { - rapidjson::Value miss_value; - miss_value.SetString( - strings::Substitute("[$0-$1]", last_version + 1, ver.first).c_str(), - missing_versions_arr.GetAllocator()); - missing_versions_arr.PushBack(miss_value, missing_versions_arr.GetAllocator()); - } - rapidjson::Value value; - std::string disk_size = - PrettyPrinter::print(rowsets[i]->rowset_meta()->total_disk_size(), TUnit::BYTES); - std::string version_str = strings::Substitute( - "[$0-$1] $2 $3 $4 $5 $6", ver.first, ver.second, rowsets[i]->num_segments(), - (delete_flags[i] ? "DELETE" : "DATA"), - SegmentsOverlapPB_Name(rowsets[i]->rowset_meta()->segments_overlap()), - rowsets[i]->rowset_id().to_string(), disk_size); - value.SetString(version_str.c_str(), version_str.length(), versions_arr.GetAllocator()); - versions_arr.PushBack(value, versions_arr.GetAllocator()); - last_version = ver.second; - } - root.AddMember("rowsets", versions_arr, root.GetAllocator()); - root.AddMember("missing_rowsets", missing_versions_arr, root.GetAllocator()); - - // print all stale rowsets' version as an array - rapidjson::Document stale_versions_arr; - stale_versions_arr.SetArray(); - for (int i = 0; i < stale_rowsets.size(); ++i) { - const Version& ver = stale_rowsets[i]->version(); - rapidjson::Value value; - std::string disk_size = PrettyPrinter::print( - stale_rowsets[i]->rowset_meta()->total_disk_size(), TUnit::BYTES); - std::string version_str = strings::Substitute( - "[$0-$1] $2 $3 $4", ver.first, ver.second, stale_rowsets[i]->num_segments(), - stale_rowsets[i]->rowset_id().to_string(), disk_size); - value.SetString(version_str.c_str(), version_str.length(), - stale_versions_arr.GetAllocator()); - stale_versions_arr.PushBack(value, stale_versions_arr.GetAllocator()); - } - root.AddMember("stale_rowsets", stale_versions_arr, root.GetAllocator()); - - // add stale version rowsets - root.AddMember("stale version path", path_arr, root.GetAllocator()); - - // to json string - rapidjson::StringBuffer strbuf; - rapidjson::PrettyWriter writer(strbuf); - root.Accept(writer); - *json_result = std::string(strbuf.GetString()); -} - -bool Tablet::do_tablet_meta_checkpoint() { - std::lock_guard store_lock(_meta_store_lock); - if (_newly_created_rowset_num == 0) { - return false; + return Status::OK(); } - if (UnixMillis() - _last_checkpoint_time < - config::tablet_meta_checkpoint_min_interval_secs * 1000 && - _newly_created_rowset_num < config::tablet_meta_checkpoint_min_new_rowsets_num) { - return false; + + Status Tablet::set_partition_id(int64_t partition_id) { + return _tablet_meta->set_partition_id(partition_id); } - // hold read-lock other than write-lock, because it will not modify meta structure - std::shared_lock rdlock(_meta_lock); - if (tablet_state() != TABLET_RUNNING) { - LOG(INFO) << "tablet is under state=" << tablet_state() - << ", not running, skip do checkpoint" - << ", tablet=" << full_name(); - return false; + TabletInfo Tablet::get_tablet_info() const { + return TabletInfo(tablet_id(), schema_hash(), tablet_uid()); } - VLOG_NOTICE << "start to do tablet meta checkpoint, tablet=" << full_name(); - save_meta(); - // if save meta successfully, then should remove the rowset meta existing in tablet - // meta from rowset meta store - for (auto& rs_meta : _tablet_meta->all_rs_metas()) { - // If we delete it from rowset manager's meta explicitly in previous checkpoint, just skip. - if (rs_meta->is_remove_from_rowset_meta()) { - continue; - } - if (RowsetMetaManager::check_rowset_meta(_data_dir->get_meta(), tablet_uid(), - rs_meta->rowset_id())) { - RowsetMetaManager::remove(_data_dir->get_meta(), tablet_uid(), rs_meta->rowset_id()); - VLOG_NOTICE << "remove rowset id from meta store because it is already persistent with " - << "tablet meta, rowset_id=" << rs_meta->rowset_id(); + + void Tablet::pick_candidate_rowsets_to_cumulative_compaction(std::vector * + candidate_rowsets) { + if (_cumulative_point == K_INVALID_CUMULATIVE_POINT) { + return; } - rs_meta->set_remove_from_rowset_meta(); + std::shared_lock rdlock(_meta_lock); + _cumulative_compaction_policy->pick_candidate_rowsets(_rs_version_map, _cumulative_point, + candidate_rowsets); } - // check _stale_rs_version_map to remove meta from rowset meta store - for (auto& rs_meta : _tablet_meta->all_stale_rs_metas()) { - // If we delete it from rowset manager's meta explicitly in previous checkpoint, just skip. - if (rs_meta->is_remove_from_rowset_meta()) { - continue; - } - if (RowsetMetaManager::check_rowset_meta(_data_dir->get_meta(), tablet_uid(), - rs_meta->rowset_id())) { - RowsetMetaManager::remove(_data_dir->get_meta(), tablet_uid(), rs_meta->rowset_id()); - VLOG_NOTICE << "remove rowset id from meta store because it is already persistent with " - << "tablet meta, rowset_id=" << rs_meta->rowset_id(); + void Tablet::pick_candidate_rowsets_to_base_compaction(vector * + candidate_rowsets) { + std::shared_lock rdlock(_meta_lock); + // FIXME(cyx): If there are delete predicate rowsets in tablet, + // remote rowsets cannot apply these delete predicate, which can cause + // incorrect query result. + for (auto& it : _rs_version_map) { + if (it.first.first < _cumulative_point && it.second->is_local()) { + candidate_rowsets->push_back(it.second); + } } - rs_meta->set_remove_from_rowset_meta(); } - _newly_created_rowset_num = 0; - _last_checkpoint_time = UnixMillis(); - return true; -} + // For http compaction action + void Tablet::get_compaction_status(std::string * json_result) { + rapidjson::Document root; + root.SetObject(); -bool Tablet::rowset_meta_is_useful(RowsetMetaSharedPtr rowset_meta) { - std::shared_lock rdlock(_meta_lock); - bool find_version = false; - for (auto& version_rowset : _rs_version_map) { - if (version_rowset.second->rowset_id() == rowset_meta->rowset_id()) { - return true; + rapidjson::Document path_arr; + path_arr.SetArray(); + + std::vector rowsets; + std::vector stale_rowsets; + std::vector delete_flags; + { + std::shared_lock rdlock(_meta_lock); + rowsets.reserve(_rs_version_map.size()); + for (auto& it : _rs_version_map) { + rowsets.push_back(it.second); + } + std::sort(rowsets.begin(), rowsets.end(), Rowset::comparator); + + stale_rowsets.reserve(_stale_rs_version_map.size()); + for (auto& it : _stale_rs_version_map) { + stale_rowsets.push_back(it.second); + } + std::sort(stale_rowsets.begin(), stale_rowsets.end(), Rowset::comparator); + + delete_flags.reserve(rowsets.size()); + for (auto& rs : rowsets) { + delete_flags.push_back(version_for_delete_predicate(rs->version())); + } + // get snapshot version path json_doc + _timestamped_version_tracker.get_stale_version_path_json_doc(path_arr); + } + rapidjson::Value cumulative_policy_type; + std::string policy_type_str = "cumulative compaction policy not initializied"; + if (_cumulative_compaction_policy != nullptr) { + policy_type_str = _cumulative_compaction_policy->name(); + } + cumulative_policy_type.SetString(policy_type_str.c_str(), policy_type_str.length(), + root.GetAllocator()); + root.AddMember("cumulative policy type", cumulative_policy_type, root.GetAllocator()); + root.AddMember("cumulative point", _cumulative_point.load(), root.GetAllocator()); + rapidjson::Value cumu_value; + std::string format_str = + ToStringFromUnixMillis(_last_cumu_compaction_failure_millis.load()); + cumu_value.SetString(format_str.c_str(), format_str.length(), root.GetAllocator()); + root.AddMember("last cumulative failure time", cumu_value, root.GetAllocator()); + rapidjson::Value base_value; + format_str = ToStringFromUnixMillis(_last_base_compaction_failure_millis.load()); + base_value.SetString(format_str.c_str(), format_str.length(), root.GetAllocator()); + root.AddMember("last base failure time", base_value, root.GetAllocator()); + rapidjson::Value cumu_success_value; + format_str = ToStringFromUnixMillis(_last_cumu_compaction_success_millis.load()); + cumu_success_value.SetString(format_str.c_str(), format_str.length(), root.GetAllocator()); + root.AddMember("last cumulative success time", cumu_success_value, root.GetAllocator()); + rapidjson::Value base_success_value; + format_str = ToStringFromUnixMillis(_last_base_compaction_success_millis.load()); + base_success_value.SetString(format_str.c_str(), format_str.length(), root.GetAllocator()); + root.AddMember("last base success time", base_success_value, root.GetAllocator()); + + // print all rowsets' version as an array + rapidjson::Document versions_arr; + rapidjson::Document missing_versions_arr; + versions_arr.SetArray(); + missing_versions_arr.SetArray(); + int64_t last_version = -1; + for (int i = 0; i < rowsets.size(); ++i) { + const Version& ver = rowsets[i]->version(); + if (ver.first != last_version + 1) { + rapidjson::Value miss_value; + miss_value.SetString( + strings::Substitute("[$0-$1]", last_version + 1, ver.first).c_str(), + missing_versions_arr.GetAllocator()); + missing_versions_arr.PushBack(miss_value, missing_versions_arr.GetAllocator()); + } + rapidjson::Value value; + std::string disk_size = PrettyPrinter::print( + rowsets[i]->rowset_meta()->total_disk_size(), TUnit::BYTES); + std::string version_str = strings::Substitute( + "[$0-$1] $2 $3 $4 $5 $6", ver.first, ver.second, rowsets[i]->num_segments(), + (delete_flags[i] ? "DELETE" : "DATA"), + SegmentsOverlapPB_Name(rowsets[i]->rowset_meta()->segments_overlap()), + rowsets[i]->rowset_id().to_string(), disk_size); + value.SetString(version_str.c_str(), version_str.length(), versions_arr.GetAllocator()); + versions_arr.PushBack(value, versions_arr.GetAllocator()); + last_version = ver.second; + } + root.AddMember("rowsets", versions_arr, root.GetAllocator()); + root.AddMember("missing_rowsets", missing_versions_arr, root.GetAllocator()); + + // print all stale rowsets' version as an array + rapidjson::Document stale_versions_arr; + stale_versions_arr.SetArray(); + for (int i = 0; i < stale_rowsets.size(); ++i) { + const Version& ver = stale_rowsets[i]->version(); + rapidjson::Value value; + std::string disk_size = PrettyPrinter::print( + stale_rowsets[i]->rowset_meta()->total_disk_size(), TUnit::BYTES); + std::string version_str = strings::Substitute( + "[$0-$1] $2 $3 $4", ver.first, ver.second, stale_rowsets[i]->num_segments(), + stale_rowsets[i]->rowset_id().to_string(), disk_size); + value.SetString(version_str.c_str(), version_str.length(), + stale_versions_arr.GetAllocator()); + stale_versions_arr.PushBack(value, stale_versions_arr.GetAllocator()); + } + root.AddMember("stale_rowsets", stale_versions_arr, root.GetAllocator()); + + // add stale version rowsets + root.AddMember("stale version path", path_arr, root.GetAllocator()); + + // to json string + rapidjson::StringBuffer strbuf; + rapidjson::PrettyWriter writer(strbuf); + root.Accept(writer); + *json_result = std::string(strbuf.GetString()); + } + + bool Tablet::do_tablet_meta_checkpoint() { + std::lock_guard store_lock(_meta_store_lock); + if (_newly_created_rowset_num == 0) { + return false; + } + if (UnixMillis() - _last_checkpoint_time < + config::tablet_meta_checkpoint_min_interval_secs * 1000 && + _newly_created_rowset_num < config::tablet_meta_checkpoint_min_new_rowsets_num) { + return false; + } + + // hold read-lock other than write-lock, because it will not modify meta structure + std::shared_lock rdlock(_meta_lock); + if (tablet_state() != TABLET_RUNNING) { + LOG(INFO) << "tablet is under state=" << tablet_state() + << ", not running, skip do checkpoint" + << ", tablet=" << full_name(); + return false; + } + VLOG_NOTICE << "start to do tablet meta checkpoint, tablet=" << full_name(); + save_meta(); + // if save meta successfully, then should remove the rowset meta existing in tablet + // meta from rowset meta store + for (auto& rs_meta : _tablet_meta->all_rs_metas()) { + // If we delete it from rowset manager's meta explicitly in previous checkpoint, just skip. + if (rs_meta->is_remove_from_rowset_meta()) { + continue; + } + if (RowsetMetaManager::check_rowset_meta(_data_dir->get_meta(), tablet_uid(), + rs_meta->rowset_id())) { + RowsetMetaManager::remove(_data_dir->get_meta(), tablet_uid(), + rs_meta->rowset_id()); + VLOG_NOTICE + << "remove rowset id from meta store because it is already persistent with " + << "tablet meta, rowset_id=" << rs_meta->rowset_id(); + } + rs_meta->set_remove_from_rowset_meta(); } - if (version_rowset.second->contains_version(rowset_meta->version())) { - find_version = true; + + // check _stale_rs_version_map to remove meta from rowset meta store + for (auto& rs_meta : _tablet_meta->all_stale_rs_metas()) { + // If we delete it from rowset manager's meta explicitly in previous checkpoint, just skip. + if (rs_meta->is_remove_from_rowset_meta()) { + continue; + } + if (RowsetMetaManager::check_rowset_meta(_data_dir->get_meta(), tablet_uid(), + rs_meta->rowset_id())) { + RowsetMetaManager::remove(_data_dir->get_meta(), tablet_uid(), + rs_meta->rowset_id()); + VLOG_NOTICE + << "remove rowset id from meta store because it is already persistent with " + << "tablet meta, rowset_id=" << rs_meta->rowset_id(); + } + rs_meta->set_remove_from_rowset_meta(); } + + _newly_created_rowset_num = 0; + _last_checkpoint_time = UnixMillis(); + return true; } - for (auto& stale_version_rowset : _stale_rs_version_map) { - if (stale_version_rowset.second->rowset_id() == rowset_meta->rowset_id()) { - return true; + + bool Tablet::rowset_meta_is_useful(RowsetMetaSharedPtr rowset_meta) { + std::shared_lock rdlock(_meta_lock); + bool find_version = false; + for (auto& version_rowset : _rs_version_map) { + if (version_rowset.second->rowset_id() == rowset_meta->rowset_id()) { + return true; + } + if (version_rowset.second->contains_version(rowset_meta->version())) { + find_version = true; + } } - if (stale_version_rowset.second->contains_version(rowset_meta->version())) { - find_version = true; + for (auto& stale_version_rowset : _stale_rs_version_map) { + if (stale_version_rowset.second->rowset_id() == rowset_meta->rowset_id()) { + return true; + } + if (stale_version_rowset.second->contains_version(rowset_meta->version())) { + find_version = true; + } } + return !find_version; } - return !find_version; -} -bool Tablet::_contains_rowset(const RowsetId rowset_id) { - for (auto& version_rowset : _rs_version_map) { - if (version_rowset.second->rowset_id() == rowset_id) { - return true; + bool Tablet::_contains_rowset(const RowsetId rowset_id) { + for (auto& version_rowset : _rs_version_map) { + if (version_rowset.second->rowset_id() == rowset_id) { + return true; + } } - } - for (auto& stale_version_rowset : _stale_rs_version_map) { - if (stale_version_rowset.second->rowset_id() == rowset_id) { - return true; + for (auto& stale_version_rowset : _stale_rs_version_map) { + if (stale_version_rowset.second->rowset_id() == rowset_id) { + return true; + } } + return false; } - return false; -} - -// need check if consecutive version missing in full report -// alter tablet will ignore this check -void Tablet::build_tablet_report_info(TTabletInfo* tablet_info, - bool enable_consecutive_missing_check) { - std::shared_lock rdlock(_meta_lock); - tablet_info->tablet_id = _tablet_meta->tablet_id(); - tablet_info->schema_hash = _tablet_meta->schema_hash(); - tablet_info->row_count = _tablet_meta->num_rows(); - tablet_info->data_size = _tablet_meta->tablet_local_size(); - - // Here we need to report to FE if there are any missing versions of tablet. - // We start from the initial version and traverse backwards until we meet a discontinuous version. - Version cversion; - Version max_version; - bool has_version_cross; - _max_continuous_version_from_beginning_unlocked(&cversion, &max_version, &has_version_cross); - // cause publish version task runs concurrently, version may be flying - // so we add a consecutive miss check to solve this problem: - // if publish version 5 arrives but version 4 flying, we may judge replica miss version - // and set version miss in tablet_info, which makes fe treat this replica as unhealth - // and lead to other problems - if (enable_consecutive_missing_check) { - if (cversion.second < max_version.second) { - if (_last_missed_version == cversion.second + 1) { - if (_last_missed_time_s - MonotonicSeconds() >= 60) { - // version missed for over 60 seconds - tablet_info->__set_version_miss(true); - _last_missed_version = -1; - _last_missed_time_s = 0; + + // need check if consecutive version missing in full report + // alter tablet will ignore this check + void Tablet::build_tablet_report_info(TTabletInfo * tablet_info, + bool enable_consecutive_missing_check) { + std::shared_lock rdlock(_meta_lock); + tablet_info->tablet_id = _tablet_meta->tablet_id(); + tablet_info->schema_hash = _tablet_meta->schema_hash(); + tablet_info->row_count = _tablet_meta->num_rows(); + tablet_info->data_size = _tablet_meta->tablet_local_size(); + + // Here we need to report to FE if there are any missing versions of tablet. + // We start from the initial version and traverse backwards until we meet a discontinuous version. + Version cversion; + Version max_version; + bool has_version_cross; + _max_continuous_version_from_beginning_unlocked(&cversion, &max_version, + &has_version_cross); + // cause publish version task runs concurrently, version may be flying + // so we add a consecutive miss check to solve this problem: + // if publish version 5 arrives but version 4 flying, we may judge replica miss version + // and set version miss in tablet_info, which makes fe treat this replica as unhealth + // and lead to other problems + if (enable_consecutive_missing_check) { + if (cversion.second < max_version.second) { + if (_last_missed_version == cversion.second + 1) { + if (_last_missed_time_s - MonotonicSeconds() >= 60) { + // version missed for over 60 seconds + tablet_info->__set_version_miss(true); + _last_missed_version = -1; + _last_missed_time_s = 0; + } + } else { + _last_missed_version = cversion.second + 1; + _last_missed_time_s = MonotonicSeconds(); } - } else { - _last_missed_version = cversion.second + 1; - _last_missed_time_s = MonotonicSeconds(); } + } else { + tablet_info->__set_version_miss(cversion.second < max_version.second); + } + // find rowset with max version + auto iter = _rs_version_map.find(max_version); + if (iter == _rs_version_map.end()) { + // If the tablet is in running state, it must not be doing schema-change. so if we can not + // access its rowsets, it means that the tablet is bad and needs to be reported to the FE + // for subsequent repairs (through the cloning task) + if (tablet_state() == TABLET_RUNNING) { + tablet_info->__set_used(false); + } + // For other states, FE knows that the tablet is in a certain change process, so here + // still sets the state to normal when reporting. Note that every task has an timeout, + // so if the task corresponding to this change hangs, when the task timeout, FE will know + // and perform state modification operations. } - } else { - tablet_info->__set_version_miss(cversion.second < max_version.second); - } - // find rowset with max version - auto iter = _rs_version_map.find(max_version); - if (iter == _rs_version_map.end()) { - // If the tablet is in running state, it must not be doing schema-change. so if we can not - // access its rowsets, it means that the tablet is bad and needs to be reported to the FE - // for subsequent repairs (through the cloning task) - if (tablet_state() == TABLET_RUNNING) { + + if (has_version_cross && tablet_state() == TABLET_RUNNING) { tablet_info->__set_used(false); } - // For other states, FE knows that the tablet is in a certain change process, so here - // still sets the state to normal when reporting. Note that every task has an timeout, - // so if the task corresponding to this change hangs, when the task timeout, FE will know - // and perform state modification operations. - } - - if (has_version_cross && tablet_state() == TABLET_RUNNING) { - tablet_info->__set_used(false); - } - - // the report version is the largest continuous version, same logic as in FE side - tablet_info->version = cversion.second; - // Useless but it is a required filed in TTabletInfo - tablet_info->version_hash = 0; - tablet_info->__set_partition_id(_tablet_meta->partition_id()); - tablet_info->__set_storage_medium(_data_dir->storage_medium()); - tablet_info->__set_version_count(_tablet_meta->version_count()); - tablet_info->__set_path_hash(_data_dir->path_hash()); - tablet_info->__set_is_in_memory(_tablet_meta->tablet_schema().is_in_memory()); - tablet_info->__set_replica_id(replica_id()); - tablet_info->__set_remote_data_size(_tablet_meta->tablet_remote_size()); -} - -// should use this method to get a copy of current tablet meta -// there are some rowset meta in local meta store and in in-memory tablet meta -// but not in tablet meta in local meta store -void Tablet::generate_tablet_meta_copy(TabletMetaSharedPtr new_tablet_meta) const { - std::shared_lock rdlock(_meta_lock); - generate_tablet_meta_copy_unlocked(new_tablet_meta); -} - -// this is a unlocked version of generate_tablet_meta_copy() -// some method already hold the _meta_lock before calling this, -// such as EngineCloneTask::_finish_clone -> tablet->revise_tablet_meta -void Tablet::generate_tablet_meta_copy_unlocked(TabletMetaSharedPtr new_tablet_meta) const { - TabletMetaPB tablet_meta_pb; - _tablet_meta->to_meta_pb(&tablet_meta_pb); - new_tablet_meta->init_from_pb(tablet_meta_pb); -} - -double Tablet::calculate_scan_frequency() { - time_t now = time(nullptr); - int64_t current_count = query_scan_count->value(); - double interval = difftime(now, _last_record_scan_count_timestamp); - double scan_frequency = (current_count - _last_record_scan_count) * 60 / interval; - if (interval >= config::tablet_scan_frequency_time_node_interval_second) { - _last_record_scan_count = current_count; - _last_record_scan_count_timestamp = now; - } - return scan_frequency; -} - -Status Tablet::prepare_compaction_and_calculate_permits(CompactionType compaction_type, - TabletSharedPtr tablet, int64_t* permits) { - std::vector compaction_rowsets; - if (compaction_type == CompactionType::CUMULATIVE_COMPACTION) { - scoped_refptr trace(new Trace); - MonotonicStopWatch watch; - watch.start(); - SCOPED_CLEANUP({ - if (watch.elapsed_time() / 1e9 > config::cumulative_compaction_trace_threshold) { - LOG(WARNING) << "Trace:" << std::endl << trace->DumpToString(Trace::INCLUDE_ALL); - } - }); - ADOPT_TRACE(trace.get()); - TRACE("create cumulative compaction"); - StorageEngine::instance()->create_cumulative_compaction(tablet, _cumulative_compaction); - DorisMetrics::instance()->cumulative_compaction_request_total->increment(1); - Status res = _cumulative_compaction->prepare_compact(); - if (!res.ok()) { - set_last_cumu_compaction_failure_time(UnixMillis()); - *permits = 0; - if (res.precise_code() != OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSION) { - DorisMetrics::instance()->cumulative_compaction_request_failed->increment(1); - return Status::InternalError("prepare cumulative compaction with err: {}", res); + // the report version is the largest continuous version, same logic as in FE side + tablet_info->version = cversion.second; + // Useless but it is a required filed in TTabletInfo + tablet_info->version_hash = 0; + tablet_info->__set_partition_id(_tablet_meta->partition_id()); + tablet_info->__set_storage_medium(_data_dir->storage_medium()); + tablet_info->__set_version_count(_tablet_meta->version_count()); + tablet_info->__set_path_hash(_data_dir->path_hash()); + tablet_info->__set_is_in_memory(_tablet_meta->tablet_schema().is_in_memory()); + tablet_info->__set_replica_id(replica_id()); + tablet_info->__set_remote_data_size(_tablet_meta->tablet_remote_size()); + } + + // should use this method to get a copy of current tablet meta + // there are some rowset meta in local meta store and in in-memory tablet meta + // but not in tablet meta in local meta store + void Tablet::generate_tablet_meta_copy(TabletMetaSharedPtr new_tablet_meta) const { + std::shared_lock rdlock(_meta_lock); + generate_tablet_meta_copy_unlocked(new_tablet_meta); + } + + // this is a unlocked version of generate_tablet_meta_copy() + // some method already hold the _meta_lock before calling this, + // such as EngineCloneTask::_finish_clone -> tablet->revise_tablet_meta + void Tablet::generate_tablet_meta_copy_unlocked(TabletMetaSharedPtr new_tablet_meta) const { + TabletMetaPB tablet_meta_pb; + _tablet_meta->to_meta_pb(&tablet_meta_pb); + new_tablet_meta->init_from_pb(tablet_meta_pb); + } + + double Tablet::calculate_scan_frequency() { + time_t now = time(nullptr); + int64_t current_count = query_scan_count->value(); + double interval = difftime(now, _last_record_scan_count_timestamp); + double scan_frequency = (current_count - _last_record_scan_count) * 60 / interval; + if (interval >= config::tablet_scan_frequency_time_node_interval_second) { + _last_record_scan_count = current_count; + _last_record_scan_count_timestamp = now; + } + return scan_frequency; + } + + Status Tablet::prepare_compaction_and_calculate_permits( + CompactionType compaction_type, TabletSharedPtr tablet, int64_t * permits) { + std::vector compaction_rowsets; + if (compaction_type == CompactionType::CUMULATIVE_COMPACTION) { + scoped_refptr trace(new Trace); + MonotonicStopWatch watch; + watch.start(); + SCOPED_CLEANUP({ + if (watch.elapsed_time() / 1e9 > config::cumulative_compaction_trace_threshold) { + LOG(WARNING) << "Trace:" << std::endl + << trace->DumpToString(Trace::INCLUDE_ALL); + } + }); + ADOPT_TRACE(trace.get()); + + TRACE("create cumulative compaction"); + StorageEngine::instance()->create_cumulative_compaction(tablet, _cumulative_compaction); + DorisMetrics::instance()->cumulative_compaction_request_total->increment(1); + Status res = _cumulative_compaction->prepare_compact(); + if (!res.ok()) { + set_last_cumu_compaction_failure_time(UnixMillis()); + *permits = 0; + if (res.precise_code() != OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSION) { + DorisMetrics::instance()->cumulative_compaction_request_failed->increment(1); + return Status::InternalError("prepare cumulative compaction with err: {}", res); + } + // return OK if OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSION, so that we don't need to + // print too much useless logs. + // And because we set permits to 0, so even if we return OK here, nothing will be done. + return Status::OK(); } - // return OK if OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSION, so that we don't need to - // print too much useless logs. - // And because we set permits to 0, so even if we return OK here, nothing will be done. - return Status::OK(); + compaction_rowsets = _cumulative_compaction->get_input_rowsets(); + } else { + DCHECK_EQ(compaction_type, CompactionType::BASE_COMPACTION); + scoped_refptr trace(new Trace); + MonotonicStopWatch watch; + watch.start(); + SCOPED_CLEANUP({ + if (watch.elapsed_time() / 1e9 > config::base_compaction_trace_threshold) { + LOG(WARNING) << "Trace:" << std::endl + << trace->DumpToString(Trace::INCLUDE_ALL); + } + }); + ADOPT_TRACE(trace.get()); + + TRACE("create base compaction"); + StorageEngine::instance()->create_base_compaction(tablet, _base_compaction); + DorisMetrics::instance()->base_compaction_request_total->increment(1); + Status res = _base_compaction->prepare_compact(); + if (!res.ok()) { + set_last_base_compaction_failure_time(UnixMillis()); + *permits = 0; + if (res.precise_code() != OLAP_ERR_BE_NO_SUITABLE_VERSION) { + DorisMetrics::instance()->base_compaction_request_failed->increment(1); + return Status::InternalError("prepare base compaction with err: {}", res); + } + // return OK if OLAP_ERR_BE_NO_SUITABLE_VERSION, so that we don't need to + // print too much useless logs. + // And because we set permits to 0, so even if we return OK here, nothing will be done. + return Status::OK(); + } + compaction_rowsets = _base_compaction->get_input_rowsets(); } - compaction_rowsets = _cumulative_compaction->get_input_rowsets(); - } else { - DCHECK_EQ(compaction_type, CompactionType::BASE_COMPACTION); - scoped_refptr trace(new Trace); - MonotonicStopWatch watch; - watch.start(); - SCOPED_CLEANUP({ - if (watch.elapsed_time() / 1e9 > config::base_compaction_trace_threshold) { - LOG(WARNING) << "Trace:" << std::endl << trace->DumpToString(Trace::INCLUDE_ALL); + *permits = 0; + for (auto rowset : compaction_rowsets) { + *permits += rowset->rowset_meta()->get_compaction_score(); + } + return Status::OK(); + } + + void Tablet::execute_compaction(CompactionType compaction_type) { + if (compaction_type == CompactionType::CUMULATIVE_COMPACTION) { + scoped_refptr trace(new Trace); + MonotonicStopWatch watch; + watch.start(); + SCOPED_CLEANUP({ + if (!config::disable_compaction_trace_log && + watch.elapsed_time() / 1e9 > config::cumulative_compaction_trace_threshold) { + LOG(WARNING) << "Trace:" << std::endl + << trace->DumpToString(Trace::INCLUDE_ALL); + } + }); + ADOPT_TRACE(trace.get()); + + TRACE("execute cumulative compaction"); + Status res = _cumulative_compaction->execute_compact(); + if (!res.ok()) { + set_last_cumu_compaction_failure_time(UnixMillis()); + DorisMetrics::instance()->cumulative_compaction_request_failed->increment(1); + LOG(WARNING) << "failed to do cumulative compaction. res=" << res + << ", tablet=" << full_name(); + return; } - }); - ADOPT_TRACE(trace.get()); + set_last_cumu_compaction_failure_time(0); + } else { + DCHECK_EQ(compaction_type, CompactionType::BASE_COMPACTION); + scoped_refptr trace(new Trace); + MonotonicStopWatch watch; + watch.start(); + SCOPED_CLEANUP({ + if (!config::disable_compaction_trace_log && + watch.elapsed_time() / 1e9 > config::base_compaction_trace_threshold) { + LOG(WARNING) << "Trace:" << std::endl + << trace->DumpToString(Trace::INCLUDE_ALL); + } + }); + ADOPT_TRACE(trace.get()); - TRACE("create base compaction"); - StorageEngine::instance()->create_base_compaction(tablet, _base_compaction); - DorisMetrics::instance()->base_compaction_request_total->increment(1); - Status res = _base_compaction->prepare_compact(); - if (!res.ok()) { - set_last_base_compaction_failure_time(UnixMillis()); - *permits = 0; - if (res.precise_code() != OLAP_ERR_BE_NO_SUITABLE_VERSION) { + TRACE("create base compaction"); + Status res = _base_compaction->execute_compact(); + if (!res.ok()) { + set_last_base_compaction_failure_time(UnixMillis()); DorisMetrics::instance()->base_compaction_request_failed->increment(1); - return Status::InternalError("prepare base compaction with err: {}", res); + LOG(WARNING) << "failed to do base compaction. res=" << res + << ", tablet=" << full_name(); + return; } - // return OK if OLAP_ERR_BE_NO_SUITABLE_VERSION, so that we don't need to - // print too much useless logs. - // And because we set permits to 0, so even if we return OK here, nothing will be done. - return Status::OK(); + set_last_base_compaction_failure_time(0); } - compaction_rowsets = _base_compaction->get_input_rowsets(); } - *permits = 0; - for (auto rowset : compaction_rowsets) { - *permits += rowset->rowset_meta()->get_compaction_score(); + + void Tablet::reset_compaction(CompactionType compaction_type) { + if (compaction_type == CompactionType::CUMULATIVE_COMPACTION) { + _cumulative_compaction.reset(); + } else { + _base_compaction.reset(); + } } - return Status::OK(); -} -void Tablet::execute_compaction(CompactionType compaction_type) { - if (compaction_type == CompactionType::CUMULATIVE_COMPACTION) { - scoped_refptr trace(new Trace); - MonotonicStopWatch watch; - watch.start(); - SCOPED_CLEANUP({ - if (!config::disable_compaction_trace_log && - watch.elapsed_time() / 1e9 > config::cumulative_compaction_trace_threshold) { - LOG(WARNING) << "Trace:" << std::endl << trace->DumpToString(Trace::INCLUDE_ALL); + Status Tablet::create_initial_rowset(const int64_t req_version) { + Status res = Status::OK(); + if (req_version < 1) { + LOG(WARNING) << "init version of tablet should at least 1. req.ver=" << req_version; + return Status::OLAPInternalError(OLAP_ERR_CE_CMD_PARAMS_ERROR); + } + Version version(0, req_version); + RowsetSharedPtr new_rowset; + do { + // there is no data in init rowset, so overlapping info is unknown. + std::unique_ptr rs_writer; + res = create_rowset_writer(version, VISIBLE, OVERLAP_UNKNOWN, + &_tablet_meta->tablet_schema(), -1, -1, &rs_writer); + + if (!res.ok()) { + LOG(WARNING) << "failed to init rowset writer for tablet " << full_name(); + break; + } + res = rs_writer->flush(); + if (!res.ok()) { + LOG(WARNING) << "failed to flush rowset writer for tablet " << full_name(); + break; } - }); - ADOPT_TRACE(trace.get()); - TRACE("execute cumulative compaction"); - Status res = _cumulative_compaction->execute_compact(); - if (!res.ok()) { - set_last_cumu_compaction_failure_time(UnixMillis()); - DorisMetrics::instance()->cumulative_compaction_request_failed->increment(1); - LOG(WARNING) << "failed to do cumulative compaction. res=" << res - << ", tablet=" << full_name(); - return; - } - set_last_cumu_compaction_failure_time(0); - } else { - DCHECK_EQ(compaction_type, CompactionType::BASE_COMPACTION); - scoped_refptr trace(new Trace); - MonotonicStopWatch watch; - watch.start(); - SCOPED_CLEANUP({ - if (!config::disable_compaction_trace_log && - watch.elapsed_time() / 1e9 > config::base_compaction_trace_threshold) { - LOG(WARNING) << "Trace:" << std::endl << trace->DumpToString(Trace::INCLUDE_ALL); + new_rowset = rs_writer->build(); + res = add_rowset(new_rowset); + if (!res.ok()) { + LOG(WARNING) << "failed to add rowset for tablet " << full_name(); + break; } - }); - ADOPT_TRACE(trace.get()); + } while (0); - TRACE("create base compaction"); - Status res = _base_compaction->execute_compact(); + // Unregister index and delete files(index and data) if failed if (!res.ok()) { - set_last_base_compaction_failure_time(UnixMillis()); - DorisMetrics::instance()->base_compaction_request_failed->increment(1); - LOG(WARNING) << "failed to do base compaction. res=" << res - << ", tablet=" << full_name(); - return; + LOG(WARNING) << "fail to create initial rowset. res=" << res + << " version=" << req_version; + StorageEngine::instance()->add_unused_rowset(new_rowset); + return res; } - set_last_base_compaction_failure_time(0); + set_cumulative_layer_point(req_version + 1); + return res; } -} -void Tablet::reset_compaction(CompactionType compaction_type) { - if (compaction_type == CompactionType::CUMULATIVE_COMPACTION) { - _cumulative_compaction.reset(); - } else { - _base_compaction.reset(); + Status Tablet::create_rowset_writer( + const Version& version, const RowsetStatePB& rowset_state, + const SegmentsOverlapPB& overlap, const doris::TabletSchema* tablet_schema, + int64_t oldest_write_timestamp, int64_t newest_write_timestamp, + std::unique_ptr* rowset_writer) { + RowsetWriterContext context; + context.version = version; + context.rowset_state = rowset_state; + context.segments_overlap = overlap; + context.oldest_write_timestamp = oldest_write_timestamp; + context.newest_write_timestamp = newest_write_timestamp; + context.tablet_schema = tablet_schema; + _init_context_common_fields(context); + return RowsetFactory::create_rowset_writer(context, rowset_writer); + } + + Status Tablet::create_rowset_writer( + const int64_t& txn_id, const PUniqueId& load_id, const RowsetStatePB& rowset_state, + const SegmentsOverlapPB& overlap, const doris::TabletSchema* tablet_schema, + std::unique_ptr* rowset_writer) { + RowsetWriterContext context; + context.txn_id = txn_id; + context.load_id = load_id; + context.rowset_state = rowset_state; + context.segments_overlap = overlap; + context.oldest_write_timestamp = -1; + context.newest_write_timestamp = -1; + context.tablet_schema = tablet_schema; + _init_context_common_fields(context); + return RowsetFactory::create_rowset_writer(context, rowset_writer); + } + + void Tablet::_init_context_common_fields(RowsetWriterContext & context) { + context.rowset_id = StorageEngine::instance()->next_rowset_id(); + context.tablet_uid = tablet_uid(); + + context.tablet_id = tablet_id(); + context.partition_id = partition_id(); + context.tablet_schema_hash = schema_hash(); + context.rowset_type = tablet_meta()->preferred_rowset_type(); + // Alpha Rowset will be removed in the future, so that if the tablet's default rowset type is + // alpah rowset, then set the newly created rowset to storage engine's default rowset. + if (context.rowset_type == ALPHA_ROWSET) { + context.rowset_type = StorageEngine::instance()->default_rowset_type(); + } + context.tablet_path = tablet_path(); + context.data_dir = data_dir(); + } + + Status Tablet::create_rowset(RowsetMetaSharedPtr rowset_meta, RowsetSharedPtr * rowset) { + return RowsetFactory::create_rowset(&tablet_schema(), tablet_path(), rowset_meta, rowset); + } + + std::shared_ptr& Tablet::get_compaction_mem_tracker( + CompactionType compaction_type) { + if (compaction_type == CompactionType::CUMULATIVE_COMPACTION) { + return _cumulative_compaction->get_mem_tracker(); + } else { + return _base_compaction->get_mem_tracker(); + } } -} -Status Tablet::create_initial_rowset(const int64_t req_version) { - Status res = Status::OK(); - if (req_version < 1) { - LOG(WARNING) << "init version of tablet should at least 1. req.ver=" << req_version; - return Status::OLAPInternalError(OLAP_ERR_CE_CMD_PARAMS_ERROR); - } - Version version(0, req_version); - RowsetSharedPtr new_rowset; - do { - // there is no data in init rowset, so overlapping info is unknown. - std::unique_ptr rs_writer; - res = create_rowset_writer(version, VISIBLE, OVERLAP_UNKNOWN, -1, -1, &rs_writer); - if (!res.ok()) { - LOG(WARNING) << "failed to init rowset writer for tablet " << full_name(); - break; + Status Tablet::cooldown() { + std::unique_lock schema_change_lock(_schema_change_lock, std::try_to_lock); + if (!schema_change_lock.owns_lock()) { + LOG(WARNING) << "Failed to own schema_change_lock. tablet=" << tablet_id(); + return Status::OLAPInternalError(OLAP_ERR_BE_TRY_BE_LOCK_ERROR); } - res = rs_writer->flush(); - if (!res.ok()) { - LOG(WARNING) << "failed to flush rowset writer for tablet " << full_name(); - break; + // Check executing serially with compaction task. + std::unique_lock base_compaction_lock(_base_compaction_lock, std::try_to_lock); + if (!base_compaction_lock.owns_lock()) { + LOG(WARNING) << "Failed to own base_compaction_lock. tablet=" << tablet_id(); + return Status::OLAPInternalError(OLAP_ERR_BE_TRY_BE_LOCK_ERROR); } - - new_rowset = rs_writer->build(); - res = add_rowset(new_rowset); - if (!res.ok()) { - LOG(WARNING) << "failed to add rowset for tablet " << full_name(); - break; + std::unique_lock cumu_compaction_lock(_cumulative_compaction_lock, std::try_to_lock); + if (!cumu_compaction_lock.owns_lock()) { + LOG(WARNING) << "Failed to own cumu_compaction_lock. tablet=" << tablet_id(); + return Status::OLAPInternalError(OLAP_ERR_BE_TRY_BE_LOCK_ERROR); } - } while (0); - - // Unregister index and delete files(index and data) if failed - if (!res.ok()) { - LOG(WARNING) << "fail to create initial rowset. res=" << res << " version=" << req_version; - StorageEngine::instance()->add_unused_rowset(new_rowset); - return res; - } - set_cumulative_layer_point(req_version + 1); - return res; -} - -Status Tablet::create_rowset_writer(const Version& version, const RowsetStatePB& rowset_state, - const SegmentsOverlapPB& overlap, - int64_t oldest_write_timestamp, int64_t newest_write_timestamp, - std::unique_ptr* rowset_writer) { - RowsetWriterContext context; - context.version = version; - context.rowset_state = rowset_state; - context.segments_overlap = overlap; - context.oldest_write_timestamp = oldest_write_timestamp; - context.newest_write_timestamp = newest_write_timestamp; - _init_context_common_fields(context); - return RowsetFactory::create_rowset_writer(context, rowset_writer); -} - -Status Tablet::create_rowset_writer(const int64_t& txn_id, const PUniqueId& load_id, - const RowsetStatePB& rowset_state, - const SegmentsOverlapPB& overlap, - std::unique_ptr* rowset_writer) { - RowsetWriterContext context; - context.txn_id = txn_id; - context.load_id = load_id; - context.rowset_state = rowset_state; - context.segments_overlap = overlap; - context.oldest_write_timestamp = -1; - context.newest_write_timestamp = -1; - _init_context_common_fields(context); - return RowsetFactory::create_rowset_writer(context, rowset_writer); -} - -void Tablet::_init_context_common_fields(RowsetWriterContext& context) { - context.rowset_id = StorageEngine::instance()->next_rowset_id(); - context.tablet_uid = tablet_uid(); - - context.tablet_id = tablet_id(); - context.partition_id = partition_id(); - context.tablet_schema_hash = schema_hash(); - context.rowset_type = tablet_meta()->preferred_rowset_type(); - // Alpha Rowset will be removed in the future, so that if the tablet's default rowset type is - // alpah rowset, then set the newly created rowset to storage engine's default rowset. - if (context.rowset_type == ALPHA_ROWSET) { - context.rowset_type = StorageEngine::instance()->default_rowset_type(); - } - context.tablet_path = tablet_path(); - context.tablet_schema = &(tablet_schema()); - context.data_dir = data_dir(); -} - -Status Tablet::create_rowset(RowsetMetaSharedPtr rowset_meta, RowsetSharedPtr* rowset) { - return RowsetFactory::create_rowset(&tablet_schema(), tablet_path(), rowset_meta, rowset); -} - -std::shared_ptr& Tablet::get_compaction_mem_tracker(CompactionType compaction_type) { - if (compaction_type == CompactionType::CUMULATIVE_COMPACTION) { - return _cumulative_compaction->get_mem_tracker(); - } else { - return _base_compaction->get_mem_tracker(); - } -} - -Status Tablet::cooldown() { - std::unique_lock schema_change_lock(_schema_change_lock, std::try_to_lock); - if (!schema_change_lock.owns_lock()) { - LOG(WARNING) << "Failed to own schema_change_lock. tablet=" << tablet_id(); - return Status::OLAPInternalError(OLAP_ERR_BE_TRY_BE_LOCK_ERROR); - } - // Check executing serially with compaction task. - std::unique_lock base_compaction_lock(_base_compaction_lock, std::try_to_lock); - if (!base_compaction_lock.owns_lock()) { - LOG(WARNING) << "Failed to own base_compaction_lock. tablet=" << tablet_id(); - return Status::OLAPInternalError(OLAP_ERR_BE_TRY_BE_LOCK_ERROR); - } - std::unique_lock cumu_compaction_lock(_cumulative_compaction_lock, std::try_to_lock); - if (!cumu_compaction_lock.owns_lock()) { - LOG(WARNING) << "Failed to own cumu_compaction_lock. tablet=" << tablet_id(); - return Status::OLAPInternalError(OLAP_ERR_BE_TRY_BE_LOCK_ERROR); - } - auto dest_fs = io::FileSystemMap::instance()->get(cooldown_resource()); - if (!dest_fs) { - return Status::OLAPInternalError(OLAP_ERR_NOT_INITED); - } - DCHECK(dest_fs->type() == io::FileSystemType::S3); - auto old_rowset = pick_cooldown_rowset(); - if (!old_rowset) { - LOG(WARNING) << "Cannot pick cooldown rowset in tablet " << tablet_id(); - return Status::OK(); - } - RowsetId new_rowset_id = StorageEngine::instance()->next_rowset_id(); - - auto start = std::chrono::steady_clock::now(); + auto dest_fs = io::FileSystemMap::instance()->get(cooldown_resource()); + if (!dest_fs) { + return Status::OLAPInternalError(OLAP_ERR_NOT_INITED); + } + DCHECK(dest_fs->type() == io::FileSystemType::S3); + auto old_rowset = pick_cooldown_rowset(); + if (!old_rowset) { + LOG(WARNING) << "Cannot pick cooldown rowset in tablet " << tablet_id(); + return Status::OK(); + } + RowsetId new_rowset_id = StorageEngine::instance()->next_rowset_id(); - RETURN_IF_ERROR(old_rowset->upload_to(reinterpret_cast(dest_fs.get()), - new_rowset_id)); + auto start = std::chrono::steady_clock::now(); - auto duration = std::chrono::duration(std::chrono::steady_clock::now() - start); - LOG(INFO) << "Upload rowset " << old_rowset->version() << " " << new_rowset_id.to_string() - << " to " << dest_fs->root_path().native() << ", tablet_id=" << tablet_id() - << ", duration=" << duration.count() << ", capacity=" << old_rowset->data_disk_size() - << ", tp=" << old_rowset->data_disk_size() / duration.count(); + RETURN_IF_ERROR(old_rowset->upload_to( + reinterpret_cast(dest_fs.get()), new_rowset_id)); - // gen a new rowset - auto new_rowset_meta = std::make_shared(*old_rowset->rowset_meta()); - new_rowset_meta->set_rowset_id(new_rowset_id); - new_rowset_meta->set_resource_id(dest_fs->resource_id()); - new_rowset_meta->set_fs(dest_fs); - new_rowset_meta->set_creation_time(time(nullptr)); - RowsetSharedPtr new_rowset; - RowsetFactory::create_rowset(&_schema, _tablet_path, std::move(new_rowset_meta), &new_rowset); + auto duration = std::chrono::duration(std::chrono::steady_clock::now() - start); + LOG(INFO) << "Upload rowset " << old_rowset->version() << " " << new_rowset_id.to_string() + << " to " << dest_fs->root_path().native() << ", tablet_id=" << tablet_id() + << ", duration=" << duration.count() + << ", capacity=" << old_rowset->data_disk_size() + << ", tp=" << old_rowset->data_disk_size() / duration.count(); - std::vector to_add {std::move(new_rowset)}; - std::vector to_delete {std::move(old_rowset)}; + // gen a new rowset + auto new_rowset_meta = std::make_shared(*old_rowset->rowset_meta()); + new_rowset_meta->set_rowset_id(new_rowset_id); + new_rowset_meta->set_resource_id(dest_fs->resource_id()); + new_rowset_meta->set_fs(dest_fs); + new_rowset_meta->set_creation_time(time(nullptr)); + RowsetSharedPtr new_rowset; + RowsetFactory::create_rowset(&_schema, _tablet_path, std::move(new_rowset_meta), + &new_rowset); - std::unique_lock meta_wlock(_meta_lock); - modify_rowsets(to_add, to_delete); - save_meta(); - return Status::OK(); -} + std::vector to_add {std::move(new_rowset)}; + std::vector to_delete {std::move(old_rowset)}; -RowsetSharedPtr Tablet::pick_cooldown_rowset() { - RowsetSharedPtr rowset; - { - std::shared_lock meta_rlock(_meta_lock); + std::unique_lock meta_wlock(_meta_lock); + modify_rowsets(to_add, to_delete); + save_meta(); + return Status::OK(); + } - // We pick the rowset with smallest start version in local. - int64_t smallest_version = std::numeric_limits::max(); - for (const auto& it : _rs_version_map) { - auto& rs = it.second; - if (rs->is_local() && rs->start_version() < smallest_version) { - smallest_version = rs->start_version(); - rowset = rs; + RowsetSharedPtr Tablet::pick_cooldown_rowset() { + RowsetSharedPtr rowset; + { + std::shared_lock meta_rlock(_meta_lock); + + // We pick the rowset with smallest start version in local. + int64_t smallest_version = std::numeric_limits::max(); + for (const auto& it : _rs_version_map) { + auto& rs = it.second; + if (rs->is_local() && rs->start_version() < smallest_version) { + smallest_version = rs->start_version(); + rowset = rs; + } } } + return rowset; } - return rowset; -} -bool Tablet::need_cooldown(int64_t* cooldown_timestamp, size_t* file_size) { - // std::shared_lock meta_rlock(_meta_lock); - if (cooldown_resource().empty()) { - VLOG_DEBUG << "tablet does not need cooldown, tablet id: " << tablet_id(); - return false; - } - auto policy = ExecEnv::GetInstance()->storage_policy_mgr()->get(cooldown_resource()); - if (!policy) { - LOG(WARNING) << "Cannot get storage policy: " << cooldown_resource(); - return false; - } - auto cooldown_ttl_sec = policy->cooldown_ttl; - auto cooldown_datetime = policy->cooldown_datetime; - RowsetSharedPtr rowset = pick_cooldown_rowset(); - if (!rowset) { - VLOG_DEBUG << "pick cooldown rowset, get null, tablet id: " << tablet_id(); - return false; - } + bool Tablet::need_cooldown(int64_t * cooldown_timestamp, size_t * file_size) { + // std::shared_lock meta_rlock(_meta_lock); + if (cooldown_resource().empty()) { + VLOG_DEBUG << "tablet does not need cooldown, tablet id: " << tablet_id(); + return false; + } + auto policy = ExecEnv::GetInstance()->storage_policy_mgr()->get(cooldown_resource()); + if (!policy) { + LOG(WARNING) << "Cannot get storage policy: " << cooldown_resource(); + return false; + } + auto cooldown_ttl_sec = policy->cooldown_ttl; + auto cooldown_datetime = policy->cooldown_datetime; + RowsetSharedPtr rowset = pick_cooldown_rowset(); + if (!rowset) { + VLOG_DEBUG << "pick cooldown rowset, get null, tablet id: " << tablet_id(); + return false; + } - int64_t oldest_cooldown_time = std::numeric_limits::max(); - if (cooldown_ttl_sec >= 0) { - oldest_cooldown_time = rowset->oldest_write_timestamp() + cooldown_ttl_sec; - } - if (cooldown_datetime > 0) { - oldest_cooldown_time = std::min(oldest_cooldown_time, cooldown_datetime); - } + int64_t oldest_cooldown_time = std::numeric_limits::max(); + if (cooldown_ttl_sec >= 0) { + oldest_cooldown_time = rowset->oldest_write_timestamp() + cooldown_ttl_sec; + } + if (cooldown_datetime > 0) { + oldest_cooldown_time = std::min(oldest_cooldown_time, cooldown_datetime); + } - int64_t newest_cooldown_time = std::numeric_limits::max(); - if (cooldown_ttl_sec >= 0) { - newest_cooldown_time = rowset->newest_write_timestamp() + cooldown_ttl_sec; - } - if (cooldown_datetime > 0) { - newest_cooldown_time = std::min(newest_cooldown_time, cooldown_datetime); - } + int64_t newest_cooldown_time = std::numeric_limits::max(); + if (cooldown_ttl_sec >= 0) { + newest_cooldown_time = rowset->newest_write_timestamp() + cooldown_ttl_sec; + } + if (cooldown_datetime > 0) { + newest_cooldown_time = std::min(newest_cooldown_time, cooldown_datetime); + } - if (oldest_cooldown_time + config::cooldown_lag_time_sec < UnixSeconds()) { - *cooldown_timestamp = oldest_cooldown_time; - VLOG_DEBUG << "tablet need cooldown, tablet id: " << tablet_id() - << " cooldown_timestamp: " << *cooldown_timestamp; - return true; + if (oldest_cooldown_time + config::cooldown_lag_time_sec < UnixSeconds()) { + *cooldown_timestamp = oldest_cooldown_time; + VLOG_DEBUG << "tablet need cooldown, tablet id: " << tablet_id() + << " cooldown_timestamp: " << *cooldown_timestamp; + return true; + } + + if (newest_cooldown_time < UnixSeconds()) { + *file_size = rowset->data_disk_size(); + VLOG_DEBUG << "tablet need cooldown, tablet id: " << tablet_id() + << " file_size: " << *file_size; + return true; + } + + VLOG_DEBUG << "tablet does not need cooldown, tablet id: " << tablet_id() + << " ttl sec: " << cooldown_ttl_sec + << " cooldown datetime: " << cooldown_datetime + << " oldest write time: " << rowset->oldest_write_timestamp() + << " newest write time: " << rowset->newest_write_timestamp(); + return false; } - if (newest_cooldown_time < UnixSeconds()) { - *file_size = rowset->data_disk_size(); - VLOG_DEBUG << "tablet need cooldown, tablet id: " << tablet_id() - << " file_size: " << *file_size; - return true; + void Tablet::remove_all_remote_rowsets() { + std::unique_lock meta_wlock(_meta_lock); + DCHECK(_state == TabletState::TABLET_SHUTDOWN); + Status st; + for (auto& it : _rs_version_map) { + auto& rs = it.second; + if (!rs->is_local()) { + st = rs->remove(); + LOG_IF(WARNING, !st.ok()) << "Failed to remove rowset " << rs->version() << " " + << rs->rowset_id().to_string() << " in tablet " + << tablet_id() << ": " << st.to_string(); + } + } + for (auto& it : _stale_rs_version_map) { + auto& rs = it.second; + if (!rs->is_local()) { + st = rs->remove(); + LOG_IF(WARNING, !st.ok()) << "Failed to remove rowset " << rs->version() << " " + << rs->rowset_id().to_string() << " in tablet " + << tablet_id() << ": " << st.to_string(); + } + } } - VLOG_DEBUG << "tablet does not need cooldown, tablet id: " << tablet_id() - << " ttl sec: " << cooldown_ttl_sec << " cooldown datetime: " << cooldown_datetime - << " oldest write time: " << rowset->oldest_write_timestamp() - << " newest write time: " << rowset->newest_write_timestamp(); - return false; -} - -void Tablet::remove_all_remote_rowsets() { - std::unique_lock meta_wlock(_meta_lock); - DCHECK(_state == TabletState::TABLET_SHUTDOWN); - Status st; - for (auto& it : _rs_version_map) { - auto& rs = it.second; - if (!rs->is_local()) { - st = rs->remove(); - LOG_IF(WARNING, !st.ok()) << "Failed to remove rowset " << rs->version() << " " - << rs->rowset_id().to_string() << " in tablet " << tablet_id() - << ": " << st.to_string(); - } - } - for (auto& it : _stale_rs_version_map) { - auto& rs = it.second; - if (!rs->is_local()) { - st = rs->remove(); - LOG_IF(WARNING, !st.ok()) << "Failed to remove rowset " << rs->version() << " " - << rs->rowset_id().to_string() << " in tablet " << tablet_id() - << ": " << st.to_string(); - } - } -} + const TabletSchema& Tablet::tablet_schema() const { + std::shared_lock wrlock(_meta_lock); + _tablet_meta->all_rs_metas(); + const RowsetMetaSharedPtr rowset_meta = + rowset_meta_with_max_schema_version(_tablet_meta->all_rs_metas()); + if (rowset_meta->tablet_schema() == nullptr) { + return _schema; + } + return *rowset_meta->tablet_schema(); + } } // namespace doris diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index 3da956cb63a7825..83f5e3d2e2e1a36 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -120,6 +120,9 @@ class Tablet : public BaseTablet { const RowsetSharedPtr rowset_with_max_version() const; + static const RowsetMetaSharedPtr rowset_meta_with_max_schema_version( + const std::vector& rowset_metas); + Status add_inc_rowset(const RowsetSharedPtr& rowset); /// Delete stale rowset by timing. This delete policy uses now() minutes /// config::tablet_rowset_expired_stale_sweep_time_sec to compute the deadline of expired rowset @@ -274,13 +277,16 @@ class Tablet : public BaseTablet { return _tablet_meta->all_beta(); } + const TabletSchema& tablet_schema() const override; + Status create_rowset_writer(const Version& version, const RowsetStatePB& rowset_state, - const SegmentsOverlapPB& overlap, int64_t oldest_write_timestamp, - int64_t newest_write_timestamp, + const SegmentsOverlapPB& overlap, const TabletSchema* tablet_schema, + int64_t oldest_write_timestamp, int64_t newest_write_timestamp, std::unique_ptr* rowset_writer); Status create_rowset_writer(const int64_t& txn_id, const PUniqueId& load_id, const RowsetStatePB& rowset_state, const SegmentsOverlapPB& overlap, + const TabletSchema* tablet_schema, std::unique_ptr* rowset_writer); Status create_rowset(RowsetMetaSharedPtr rowset_meta, RowsetSharedPtr* rowset); diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp index bd3161fd306cebe..5802d43c39028a1 100644 --- a/be/src/olap/tablet_meta.cpp +++ b/be/src/olap/tablet_meta.cpp @@ -138,8 +138,13 @@ TabletMeta::TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id bool has_bf_columns = false; for (TColumn tcolumn : tablet_schema.columns) { ColumnPB* column = schema->add_column(); - uint32_t unique_id = col_ordinal_to_unique_id.at(col_ordinal++); - _init_column_from_tcolumn(unique_id, tcolumn, column); + uint32_t unique_id = -1; + if (tcolumn.col_unique_id >= 0) { + unique_id = tcolumn.col_unique_id; + } else { + unique_id = col_ordinal_to_unique_id.at(col_ordinal++); + } + init_column_from_tcolumn(unique_id, tcolumn, column); if (column->is_key()) { ++key_count; @@ -199,11 +204,12 @@ TabletMeta::TabletMeta(const TabletMeta& b) _storage_medium(b._storage_medium), _cooldown_resource(b._cooldown_resource) {}; -void TabletMeta::_init_column_from_tcolumn(uint32_t unique_id, const TColumn& tcolumn, - ColumnPB* column) { +void TabletMeta::init_column_from_tcolumn(uint32_t unique_id, const TColumn& tcolumn, + ColumnPB* column) { column->set_unique_id(unique_id); + column->set_col_unique_id(tcolumn.col_unique_id); column->set_name(tcolumn.column_name); - column->set_has_bitmap_index(false); + column->set_has_bitmap_index(tcolumn.has_bitmap_index); string data_type; EnumToString(TPrimitiveType, tcolumn.column_type.type, data_type); column->set_type(data_type); @@ -242,7 +248,7 @@ void TabletMeta::_init_column_from_tcolumn(uint32_t unique_id, const TColumn& tc } if (tcolumn.column_type.type == TPrimitiveType::ARRAY) { ColumnPB* children_column = column->add_children_columns(); - _init_column_from_tcolumn(0, tcolumn.children_column[0], children_column); + init_column_from_tcolumn(0, tcolumn.children_column[0], children_column); } } diff --git a/be/src/olap/tablet_meta.h b/be/src/olap/tablet_meta.h index e0010d8f415191a..5e79a4c6f5ef3bc 100644 --- a/be/src/olap/tablet_meta.h +++ b/be/src/olap/tablet_meta.h @@ -148,6 +148,7 @@ class TabletMeta { TabletSchema* mutable_tablet_schema(); const std::vector& all_rs_metas() const; + std::vector& all_mutable_rs_metas(); Status add_rs_meta(const RowsetMetaSharedPtr& rs_meta); void delete_rs_meta_by_version(const Version& version, std::vector* deleted_rs_metas); @@ -198,10 +199,11 @@ class TabletMeta { << _cooldown_resource << " to " << resource; _cooldown_resource = std::move(resource); } + static void init_column_from_tcolumn(uint32_t unique_id, const TColumn& tcolumn, + ColumnPB* column); private: Status _save_meta(DataDir* data_dir); - void _init_column_from_tcolumn(uint32_t unique_id, const TColumn& tcolumn, ColumnPB* column); // _del_pred_array is ignored to compare. friend bool operator==(const TabletMeta& a, const TabletMeta& b); @@ -360,6 +362,10 @@ inline const std::vector& TabletMeta::all_rs_metas() const return _rs_metas; } +inline std::vector& TabletMeta::all_mutable_rs_metas() { + return _rs_metas; +} + inline const std::vector& TabletMeta::all_stale_rs_metas() const { return _stale_rs_metas; } diff --git a/be/src/olap/tablet_schema.cpp b/be/src/olap/tablet_schema.cpp index cda1370b7ff0bc1..c3ceb2b331c6142 100644 --- a/be/src/olap/tablet_schema.cpp +++ b/be/src/olap/tablet_schema.cpp @@ -17,6 +17,7 @@ #include "olap/tablet_schema.h" +#include "gen_cpp/descriptors.pb.h" #include "tablet_meta.h" #include "vec/aggregate_functions/aggregate_function_reader.h" #include "vec/aggregate_functions/aggregate_function_simple_factory.h" @@ -307,8 +308,24 @@ TabletColumn::TabletColumn(FieldAggregationMethod agg, FieldType filed_type, boo _length = length; } +TabletColumn::TabletColumn(const ColumnPB& column) { + init_from_pb(column); +} + +TabletColumn::TabletColumn(const TColumn& column) { + init_from_thrift(column); +} + +void TabletColumn::init_from_thrift(const TColumn& tcolumn) { + _unique_id = tcolumn.col_unique_id; + ColumnPB column_pb; + TabletMeta::init_column_from_tcolumn(_unique_id, tcolumn, &column_pb); + init_from_pb(column_pb); +} + void TabletColumn::init_from_pb(const ColumnPB& column) { _unique_id = column.unique_id(); + _col_unique_id = column.col_unique_id(); _col_name = column.name(); _type = TabletColumn::get_field_type_by_string(column.type()); _is_key = column.is_key(); @@ -358,9 +375,10 @@ void TabletColumn::init_from_pb(const ColumnPB& column) { } } -void TabletColumn::to_schema_pb(ColumnPB* column) { +void TabletColumn::to_schema_pb(ColumnPB* column) const { column->set_unique_id(_unique_id); column->set_name(_col_name); + column->set_col_unique_id(_col_unique_id); column->set_type(get_string_by_field_type(_type)); column->set_is_key(_is_key); column->set_is_nullable(_is_nullable); @@ -423,6 +441,28 @@ vectorized::AggregateFunctionPtr TabletColumn::get_aggregate_function( agg_name, argument_types, {}, argument_types.back()->is_nullable()); } +void TabletSchema::append_column(TabletColumn column) { + if (column.is_key()) { + _num_key_columns++; + } + if (column.is_nullable()) { + _num_null_columns++; + } + _field_name_to_index[column.name()] = _num_columns; + _field_id_to_index[column.col_unique_id()] = _num_columns; + _cols.push_back(std::move(column)); + _num_columns++; +} + +void TabletSchema::clear_columns() { + _field_name_to_index.clear(); + _field_id_to_index.clear(); + _num_columns = 0; + _num_null_columns = 0; + _num_key_columns = 0; + _cols.clear(); +} + void TabletSchema::init_from_pb(const TabletSchemaPB& schema) { _keys_type = schema.keys_type(); _num_columns = 0; @@ -440,6 +480,9 @@ void TabletSchema::init_from_pb(const TabletSchemaPB& schema) { _num_null_columns++; } _field_name_to_index[column.name()] = _num_columns; + if (column.col_unique_id() >= 0) { + _field_id_to_index[column.col_unique_id()] = _num_columns; + } _cols.emplace_back(std::move(column)); _num_columns++; } @@ -460,27 +503,87 @@ void TabletSchema::init_from_pb(const TabletSchemaPB& schema) { _sort_type = schema.sort_type(); _sort_col_num = schema.sort_col_num(); _compression_type = schema.compression_type(); + _schema_version = schema.schema_version(); } -void TabletSchema::to_schema_pb(TabletSchemaPB* tablet_meta_pb) { - tablet_meta_pb->set_keys_type(_keys_type); +void TabletSchema::build_current_tablet_schema(int64_t index_id, + const POlapTableSchemaParam& ptable_schema_param, + const TabletSchema& ori_tablet_schema) { + // copy from ori_tablet_schema + _keys_type = ori_tablet_schema.keys_type(); + _num_short_key_columns = ori_tablet_schema.num_short_key_columns(); + _num_rows_per_row_block = ori_tablet_schema.num_rows_per_row_block(); + _compress_kind = ori_tablet_schema.compress_kind(); + + // todo(yixiu): unique_id + _next_column_unique_id = ori_tablet_schema.next_column_unique_id(); + _is_in_memory = ori_tablet_schema.is_in_memory(); + _delete_sign_idx = ori_tablet_schema.delete_sign_idx(); + _sequence_col_idx = ori_tablet_schema.sequence_col_idx(); + _sort_type = ori_tablet_schema.sort_type(); + _sort_col_num = ori_tablet_schema.sort_col_num(); + + // copy from table_schema_param + _num_columns = 0; + _num_key_columns = 0; + _num_null_columns = 0; + bool has_bf_columns = false; + _cols.clear(); + _field_name_to_index.clear(); + _field_id_to_index.clear(); + + for (const POlapTableIndexSchema& index : ptable_schema_param.indexes()) { + if (index.id() == index_id) { + for (auto& pcolumn : index.columns_desc()) { + TabletColumn column; + column.init_from_pb(pcolumn); + if (column.is_key()) { + _num_key_columns++; + } + if (column.is_nullable()) { + _num_null_columns++; + } + if (column.is_bf_column()) { + has_bf_columns = true; + } + _field_name_to_index[column.name()] = _num_columns; + _field_id_to_index[column.col_unique_id()] = _num_columns; + _cols.emplace_back(std::move(column)); + _num_columns++; + } + break; + } + } + if (has_bf_columns) { + _has_bf_fpp = true; + _bf_fpp = ori_tablet_schema.bloom_filter_fpp(); + } else { + _has_bf_fpp = false; + _bf_fpp = BLOOM_FILTER_DEFAULT_FPP; + } + _schema_version = ptable_schema_param.version(); +} + +void TabletSchema::to_schema_pb(TabletSchemaPB* tablet_schema_pb) const { + tablet_schema_pb->set_keys_type(_keys_type); for (auto& col : _cols) { - ColumnPB* column = tablet_meta_pb->add_column(); + ColumnPB* column = tablet_schema_pb->add_column(); col.to_schema_pb(column); } - tablet_meta_pb->set_num_short_key_columns(_num_short_key_columns); - tablet_meta_pb->set_num_rows_per_row_block(_num_rows_per_row_block); - tablet_meta_pb->set_compress_kind(_compress_kind); + tablet_schema_pb->set_num_short_key_columns(_num_short_key_columns); + tablet_schema_pb->set_num_rows_per_row_block(_num_rows_per_row_block); + tablet_schema_pb->set_compress_kind(_compress_kind); if (_has_bf_fpp) { - tablet_meta_pb->set_bf_fpp(_bf_fpp); - } - tablet_meta_pb->set_next_column_unique_id(_next_column_unique_id); - tablet_meta_pb->set_is_in_memory(_is_in_memory); - tablet_meta_pb->set_delete_sign_idx(_delete_sign_idx); - tablet_meta_pb->set_sequence_col_idx(_sequence_col_idx); - tablet_meta_pb->set_sort_type(_sort_type); - tablet_meta_pb->set_sort_col_num(_sort_col_num); - tablet_meta_pb->set_compression_type(_compression_type); + tablet_schema_pb->set_bf_fpp(_bf_fpp); + } + tablet_schema_pb->set_next_column_unique_id(_next_column_unique_id); + tablet_schema_pb->set_is_in_memory(_is_in_memory); + tablet_schema_pb->set_delete_sign_idx(_delete_sign_idx); + tablet_schema_pb->set_sequence_col_idx(_sequence_col_idx); + tablet_schema_pb->set_sort_type(_sort_type); + tablet_schema_pb->set_sort_col_num(_sort_col_num); + tablet_schema_pb->set_schema_version(_schema_version); + tablet_schema_pb->set_compression_type(_compression_type); } uint32_t TabletSchema::mem_size() const { @@ -511,6 +614,11 @@ int32_t TabletSchema::field_index(const std::string& field_name) const { return (found == _field_name_to_index.end()) ? -1 : found->second; } +int32_t TabletSchema::field_index(int32_t col_unique_od) const { + const auto& found = _field_id_to_index.find(col_unique_od); + return (found == _field_id_to_index.end()) ? -1 : found->second; +} + const std::vector& TabletSchema::columns() const { return _cols; } diff --git a/be/src/olap/tablet_schema.h b/be/src/olap/tablet_schema.h index f0a5b72157f8d98..1ae27b8cb34319f 100644 --- a/be/src/olap/tablet_schema.h +++ b/be/src/olap/tablet_schema.h @@ -31,18 +31,24 @@ namespace vectorized { class Block; } +class POlapTableSchemaParam; + class TabletColumn { public: TabletColumn(); + TabletColumn(const ColumnPB& column); + TabletColumn(const TColumn& column); TabletColumn(FieldAggregationMethod agg, FieldType type); TabletColumn(FieldAggregationMethod agg, FieldType filed_type, bool is_nullable); TabletColumn(FieldAggregationMethod agg, FieldType filed_type, bool is_nullable, int32_t unique_id, size_t length); void init_from_pb(const ColumnPB& column); - void to_schema_pb(ColumnPB* column); + void init_from_thrift(const TColumn& column); + void to_schema_pb(ColumnPB* column) const; uint32_t mem_size() const; int32_t unique_id() const { return _unique_id; } + int32_t col_unique_id() const { return _col_unique_id; } std::string name() const { return _col_name; } void set_name(std::string col_name) { _col_name = col_name; } FieldType type() const { return _type; } @@ -114,6 +120,7 @@ class TabletColumn { TabletColumn* _parent = nullptr; std::vector _sub_columns; uint32_t _sub_column_count = 0; + int32_t _col_unique_id = -1; }; bool operator==(const TabletColumn& a, const TabletColumn& b); @@ -126,11 +133,13 @@ class TabletSchema { // void create_from_pb(const TabletSchemaPB& schema, TabletSchema* tablet_schema). TabletSchema() = default; void init_from_pb(const TabletSchemaPB& schema); - void to_schema_pb(TabletSchemaPB* tablet_meta_pb); + void to_schema_pb(TabletSchemaPB* tablet_meta_pb) const; + void append_column(TabletColumn column); uint32_t mem_size() const; size_t row_size() const; int32_t field_index(const std::string& field_name) const; + int32_t field_index(int32_t col_unique_id) const; const TabletColumn& column(size_t ordinal) const; const std::vector& columns() const; size_t num_columns() const { return _num_columns; } @@ -143,6 +152,7 @@ class TabletSchema { size_t sort_col_num() const { return _sort_col_num; } CompressKind compress_kind() const { return _compress_kind; } size_t next_column_unique_id() const { return _next_column_unique_id; } + bool has_bf_fpp() const { return _has_bf_fpp; } double bloom_filter_fpp() const { return _bf_fpp; } bool is_in_memory() const { return _is_in_memory; } void set_is_in_memory(bool is_in_memory) { _is_in_memory = is_in_memory; } @@ -152,11 +162,17 @@ class TabletSchema { int32_t sequence_col_idx() const { return _sequence_col_idx; } segment_v2::CompressionTypePB compression_type() const { return _compression_type; } + int32_t schema_version() const { return _schema_version; } + void clear_columns(); vectorized::Block create_block( const std::vector& return_columns, const std::unordered_set* tablet_columns_need_convert_null = nullptr) const; vectorized::Block create_block() const; + void build_current_tablet_schema(int64_t index_id, + const POlapTableSchemaParam& ptable_schema_param, + const TabletSchema& out_tablet_schema); + private: // Only for unit test. void init_field_index_for_test(); @@ -170,6 +186,7 @@ class TabletSchema { size_t _sort_col_num = 0; std::vector _cols; std::unordered_map _field_name_to_index; + std::unordered_map _field_id_to_index; size_t _num_columns = 0; size_t _num_key_columns = 0; size_t _num_null_columns = 0; @@ -184,6 +201,7 @@ class TabletSchema { bool _is_in_memory = false; int32_t _delete_sign_idx = -1; int32_t _sequence_col_idx = -1; + int32_t _schema_version = -1; }; bool operator==(const TabletSchema& a, const TabletSchema& b); diff --git a/be/src/runtime/descriptors.cpp b/be/src/runtime/descriptors.cpp index 4b81fc2cfeb323e..e035068ef2d40c9 100644 --- a/be/src/runtime/descriptors.cpp +++ b/be/src/runtime/descriptors.cpp @@ -55,6 +55,7 @@ SlotDescriptor::SlotDescriptor(const TSlotDescriptor& tdesc) _tuple_offset(tdesc.byteOffset), _null_indicator_offset(tdesc.nullIndicatorByte, tdesc.nullIndicatorBit), _col_name(tdesc.colName), + _col_unique_id(tdesc.col_unique_id), _slot_idx(tdesc.slotIdx), _slot_size(_type.get_slot_size()), _field_idx(-1), @@ -68,6 +69,7 @@ SlotDescriptor::SlotDescriptor(const PSlotDescriptor& pdesc) _tuple_offset(pdesc.byte_offset()), _null_indicator_offset(pdesc.null_indicator_byte(), pdesc.null_indicator_bit()), _col_name(pdesc.col_name()), + _col_unique_id(-1), _slot_idx(pdesc.slot_idx()), _slot_size(_type.get_slot_size()), _field_idx(-1), diff --git a/be/src/runtime/descriptors.h b/be/src/runtime/descriptors.h index ee18f8a45068751..85f5bea18a9f1a1 100644 --- a/be/src/runtime/descriptors.h +++ b/be/src/runtime/descriptors.h @@ -113,6 +113,8 @@ class SlotDescriptor { doris::vectorized::DataTypePtr get_data_type_ptr() const; + int32_t col_unique_id() const { return _col_unique_id; } + private: friend class DescriptorTbl; friend class TupleDescriptor; @@ -127,6 +129,8 @@ class SlotDescriptor { const NullIndicatorOffset _null_indicator_offset; const std::string _col_name; + const int32_t _col_unique_id; + // the idx of the slot in the tuple descriptor (0-based). // this is provided by the FE const int _slot_idx; diff --git a/be/src/runtime/tablets_channel.cpp b/be/src/runtime/tablets_channel.cpp index 9733ff0bc7d1448..2ee5ea2dcdebecc 100644 --- a/be/src/runtime/tablets_channel.cpp +++ b/be/src/runtime/tablets_channel.cpp @@ -225,6 +225,7 @@ Status TabletsChannel::_open_all_writers(const PTabletWriterOpenRequest& request } for (auto& tablet : request.tablets()) { WriteRequest wrequest; + wrequest.index_id = request.index_id(); wrequest.tablet_id = tablet.tablet_id(); wrequest.schema_hash = schema_hash; wrequest.write_type = WriteType::LOAD; @@ -234,6 +235,7 @@ Status TabletsChannel::_open_all_writers(const PTabletWriterOpenRequest& request wrequest.tuple_desc = _tuple_desc; wrequest.slots = index_slots; wrequest.is_high_priority = _is_high_priority; + wrequest.ptable_schema_param = request.schema(); DeltaWriter* writer = nullptr; auto st = DeltaWriter::open(&wrequest, &writer, _is_vec); diff --git a/be/src/runtime/tablets_channel.h b/be/src/runtime/tablets_channel.h index 5ad1b47027b8968..ed703f4c6fdb071 100644 --- a/be/src/runtime/tablets_channel.h +++ b/be/src/runtime/tablets_channel.h @@ -204,6 +204,7 @@ Status TabletsChannel::add_batch(const TabletWriterAddRequest& request, if constexpr (std::is_same_v) { return RowBatch(*_row_desc, request.row_batch()); } else { + LOG(INFO) << "2 block columns: " << vectorized::Block(request.block()).columns(); return vectorized::Block(request.block()); } }; diff --git a/be/src/vec/exec/volap_scanner.cpp b/be/src/vec/exec/volap_scanner.cpp index 21c94c0e73296a6..45ba3d30da83a49 100644 --- a/be/src/vec/exec/volap_scanner.cpp +++ b/be/src/vec/exec/volap_scanner.cpp @@ -73,6 +73,18 @@ Status VOlapScanner::prepare( LOG(WARNING) << ss.str(); return Status::InternalError(ss.str()); } + _tablet_schema = _tablet->tablet_schema(); + if (!_parent->_olap_scan_node.columns_desc.empty() && + _parent->_olap_scan_node.columns_desc[0].col_unique_id >= 0) { + // Originally scanner get TabletSchema from tablet object in BE. + // To support lightweight schema change for adding / dropping columns, + // tabletschema is bounded to rowset and tablet's schema maybe outdated, + // so we have to use schema from a query plan witch FE puts it in query plans. + _tablet_schema.clear_columns(); + for (const auto& column_desc : _parent->_olap_scan_node.columns_desc) { + _tablet_schema.append_column(TabletColumn(column_desc)); + } + } { std::shared_lock rdlock(_tablet->get_header_lock()); const RowsetSharedPtr rowset = _tablet->rowset_with_max_version(); @@ -155,6 +167,7 @@ Status VOlapScanner::_init_tablet_reader_params( RETURN_IF_ERROR(_init_return_columns(!_tablet_reader_params.direct_mode)); _tablet_reader_params.tablet = _tablet; + _tablet_reader_params.tablet_schema = &_tablet_schema; _tablet_reader_params.reader_type = READER_QUERY; _tablet_reader_params.aggregation = _aggregation; _tablet_reader_params.version = Version(0, _version); @@ -191,11 +204,11 @@ Status VOlapScanner::_init_tablet_reader_params( _tablet_reader_params.return_columns = _return_columns; } else { // we need to fetch all key columns to do the right aggregation on storage engine side. - for (size_t i = 0; i < _tablet->num_key_columns(); ++i) { + for (size_t i = 0; i < _tablet_schema.num_key_columns(); ++i) { _tablet_reader_params.return_columns.push_back(i); } for (auto index : _return_columns) { - if (_tablet->tablet_schema().column(index).is_key()) { + if (_tablet_schema.column(index).is_key()) { continue; } else { _tablet_reader_params.return_columns.push_back(index); @@ -220,7 +233,9 @@ Status VOlapScanner::_init_return_columns(bool need_seq_col) { if (!slot->is_materialized()) { continue; } - int32_t index = _tablet->field_index(slot->col_name()); + int32_t index = slot->col_unique_id() >= 0 + ? _tablet_schema.field_index(slot->col_unique_id()) + : _tablet_schema.field_index(slot->col_name()); if (index < 0) { std::stringstream ss; ss << "field name is invalid. field=" << slot->col_name(); @@ -228,21 +243,21 @@ Status VOlapScanner::_init_return_columns(bool need_seq_col) { return Status::InternalError(ss.str()); } _return_columns.push_back(index); - if (slot->is_nullable() && !_tablet->tablet_schema().column(index).is_nullable()) + if (slot->is_nullable() && !_tablet_schema.column(index).is_nullable()) _tablet_columns_convert_to_null_set.emplace(index); } // expand the sequence column - if (_tablet->tablet_schema().has_sequence_col() && need_seq_col) { + if (_tablet_schema.has_sequence_col() && need_seq_col) { bool has_replace_col = false; for (auto col : _return_columns) { - if (_tablet->tablet_schema().column(col).aggregation() == + if (_tablet_schema.column(col).aggregation() == FieldAggregationMethod::OLAP_FIELD_AGGREGATION_REPLACE) { has_replace_col = true; break; } } - if (auto sequence_col_idx = _tablet->tablet_schema().sequence_col_idx(); + if (auto sequence_col_idx = _tablet_schema.sequence_col_idx(); has_replace_col && std::find(_return_columns.begin(), _return_columns.end(), sequence_col_idx) == _return_columns.end()) { _return_columns.push_back(sequence_col_idx); diff --git a/be/src/vec/exec/volap_scanner.h b/be/src/vec/exec/volap_scanner.h index 9733fa8227b04b6..0893566b3443789 100644 --- a/be/src/vec/exec/volap_scanner.h +++ b/be/src/vec/exec/volap_scanner.h @@ -140,6 +140,8 @@ class VOlapScanner { VExprContext* _vconjunct_ctx = nullptr; bool _need_to_close = false; + + TabletSchema _tablet_schema; }; } // namespace vectorized diff --git a/be/src/vec/olap/block_reader.cpp b/be/src/vec/olap/block_reader.cpp index 681b6d4e55da7f2..ab187367f9414ac 100644 --- a/be/src/vec/olap/block_reader.cpp +++ b/be/src/vec/olap/block_reader.cpp @@ -79,7 +79,7 @@ void BlockReader::_init_agg_state(const ReaderParams& read_params) { _stored_has_null_tag.resize(_stored_data_columns.size()); _stored_has_string_tag.resize(_stored_data_columns.size()); - auto& tablet_schema = tablet()->tablet_schema(); + auto& tablet_schema = *_tablet_schema; for (auto idx : _agg_columns_idx) { AggregateFunctionPtr function = tablet_schema diff --git a/be/src/vec/olap/vcollect_iterator.cpp b/be/src/vec/olap/vcollect_iterator.cpp index 3d73ab8860d23e3..a87db2bf54553d0 100644 --- a/be/src/vec/olap/vcollect_iterator.cpp +++ b/be/src/vec/olap/vcollect_iterator.cpp @@ -55,7 +55,7 @@ Status VCollectIterator::add_child(RowsetReaderSharedPtr rs_reader) { // then merged with the base rowset. Status VCollectIterator::build_heap(std::vector& rs_readers) { DCHECK(rs_readers.size() == _children.size()); - _skip_same = _reader->_tablet->tablet_schema().keys_type() == KeysType::UNIQUE_KEYS; + _skip_same = _reader->_tablet_schema->keys_type() == KeysType::UNIQUE_KEYS; if (_children.empty()) { _inner_iter.reset(nullptr); return Status::OK(); diff --git a/be/src/vec/olap/vcollect_iterator.h b/be/src/vec/olap/vcollect_iterator.h index 6cae36dcc328507..4c8002d88520095 100644 --- a/be/src/vec/olap/vcollect_iterator.h +++ b/be/src/vec/olap/vcollect_iterator.h @@ -73,7 +73,7 @@ class VCollectIterator { // then merged with other rowset readers. class LevelIterator { public: - LevelIterator(TabletReader* reader) : _schema(reader->tablet()->tablet_schema()) {}; + LevelIterator(TabletReader* reader) : _schema(reader->tablet_schema()) {}; virtual Status init() = 0; diff --git a/be/test/olap/test_data/header_without_inc_rs.txt b/be/test/olap/test_data/header_without_inc_rs.txt index bde8d7f4d58d493..8a47a4e3f70c02b 100644 --- a/be/test/olap/test_data/header_without_inc_rs.txt +++ b/be/test/olap/test_data/header_without_inc_rs.txt @@ -19,7 +19,8 @@ "is_nullable": false, "length": 8, "index_length": 8, - "visible": true + "visible": true, + "col_unique_id": -1 }, { "unique_id": 1, @@ -31,7 +32,8 @@ "default_value": "MA==", "length": 16387, "index_length": 16, - "visible": true + "visible": true, + "col_unique_id": -1 }, { "unique_id": 2, @@ -42,7 +44,8 @@ "is_nullable": false, "length": 4, "index_length": 4, - "visible": true + "visible": true, + "col_unique_id": -1 } ], "num_short_key_columns": 1, @@ -54,7 +57,8 @@ "sequence_col_idx": -1, "sort_type": "LEXICAL", "sort_col_num": 0, - "compression_type": "LZ4F" + "compression_type": "LZ4F", + "schema_version": 0 }, "rs_metas": [ { diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java b/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java index 1a5d6da4ce630f9..7ecc5eb1ffc3303 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java @@ -42,8 +42,10 @@ public final class FeMetaVersion { public static final int VERSION_110 = 110; // add catalog PrivTable in PaloAuth to support unified privilege management public static final int VERSION_111 = 111; + // add row policy and add maxColUniqueId for olapTable + public static final int VERSION_112 = 112; // note: when increment meta version, should assign the latest version to VERSION_CURRENT - public static final int VERSION_CURRENT = VERSION_111; + public static final int VERSION_CURRENT = VERSION_112; // all logs meta version should >= the minimum version, so that we could remove many if clause, for example // if (FE_METAVERSION < VERSION_94) ... diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java index 89dce3e16991659..9cfe420f34f3d08 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java @@ -111,6 +111,10 @@ public JobState getJobState() { return jobState; } + public void setJobState(JobState jobState) { + this.jobState = jobState; + } + public JobType getType() { return type; } @@ -143,6 +147,10 @@ public long getFinishedTimeMs() { return finishedTimeMs; } + public void setFinishedTimeMs(long finishedTimeMs) { + this.finishedTimeMs = finishedTimeMs; + } + /** * The keyword 'synchronized' only protects 2 methods: * run() and cancel() diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java index 79ec2d410754845..3c36028ad0bf8d6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java @@ -385,7 +385,7 @@ protected void runWaitingTxnJob() throws AlterCancelException { AlterReplicaTask rollupTask = new AlterReplicaTask(rollupReplica.getBackendId(), dbId, tableId, partitionId, rollupIndexId, baseIndexId, rollupTabletId, baseTabletId, rollupReplica.getId(), rollupSchemaHash, baseSchemaHash, visibleVersion, jobId, - JobType.ROLLUP, defineExprs, descTable); + JobType.ROLLUP, defineExprs, descTable, null); rollupBatchTask.addTask(rollupTask); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index 574c90880388d7b..c5b51518828b354 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -53,6 +53,7 @@ import org.apache.doris.catalog.Replica.ReplicaState; import org.apache.doris.catalog.ReplicaAllocation; import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.TableIf.TableType; import org.apache.doris.catalog.Tablet; import org.apache.doris.catalog.TabletMeta; import org.apache.doris.common.AnalysisException; @@ -70,6 +71,7 @@ import org.apache.doris.common.util.Util; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.persist.RemoveAlterJobV2OperationLog; +import org.apache.doris.persist.TableAddOrDropColumnsInfo; import org.apache.doris.qe.ConnectContext; import org.apache.doris.task.AgentBatchTask; import org.apache.doris.task.AgentTaskExecutor; @@ -101,6 +103,7 @@ import java.util.Set; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.function.IntSupplier; public class SchemaChangeHandler extends AlterHandler { private static final Logger LOG = LogManager.getLogger(SchemaChangeHandler.class); @@ -125,8 +128,16 @@ public SchemaChangeHandler() { super("schema change", Config.default_schema_change_scheduler_interval_millisecond); } - private void processAddColumn(AddColumnClause alterClause, OlapTable olapTable, - Map> indexSchemaMap) throws DdlException { + /** + * @param alterClause + * @param olapTable + * @param indexSchemaMap + * @param colUniqueIdSupplier for multi add columns clause, we need stash middle state of maxColUniqueId + * @return true: can light schema change, false: cannot light schema change + * @throws DdlException + */ + private boolean processAddColumn(AddColumnClause alterClause, OlapTable olapTable, + Map> indexSchemaMap, IntSupplier colUniqueIdSupplier) throws DdlException { Column column = alterClause.getColumn(); ColumnPosition columnPos = alterClause.getColPos(); String targetIndexName = alterClause.getRollupName(); @@ -142,12 +153,18 @@ private void processAddColumn(AddColumnClause alterClause, OlapTable olapTable, } Set newColNameSet = Sets.newHashSet(column.getName()); - addColumnInternal(olapTable, column, columnPos, targetIndexId, baseIndexId, - indexSchemaMap, newColNameSet); + + //only new table generate ColUniqueId, exist table do not. + if (olapTable.getMaxColUniqueId() > Column.COLUMN_UNIQUE_ID_INIT_VALUE) { + column.setUniqueId(colUniqueIdSupplier.getAsInt()); + } + + return addColumnInternal(olapTable, column, columnPos, targetIndexId, baseIndexId, indexSchemaMap, + newColNameSet, false); } - private void processAddColumn(AddColumnClause alterClause, - Table externalTable, List newSchema) throws DdlException { + private void processAddColumn(AddColumnClause alterClause, Table externalTable, List newSchema) + throws DdlException { Column column = alterClause.getColumn(); ColumnPosition columnPos = alterClause.getColPos(); Set newColNameSet = Sets.newHashSet(column.getName()); @@ -155,8 +172,8 @@ private void processAddColumn(AddColumnClause alterClause, addColumnInternal(column, columnPos, newSchema, newColNameSet); } - private void processAddColumns(AddColumnsClause alterClause, - Table externalTable, List newSchema) throws DdlException { + private void processAddColumns(AddColumnsClause alterClause, Table externalTable, List newSchema) + throws DdlException { List columns = alterClause.getColumns(); Set newColNameSet = Sets.newHashSet(); for (Column column : alterClause.getColumns()) { @@ -168,8 +185,18 @@ private void processAddColumns(AddColumnsClause alterClause, } } - private void processAddColumns(AddColumnsClause alterClause, OlapTable olapTable, - Map> indexSchemaMap) throws DdlException { + /** + * @param alterClause + * @param olapTable + * @param indexSchemaMap + * @param ignoreSameColumn + * @param colUniqueIdSupplier for multi add columns clause, we need stash middle state of maxColUniqueId + * @return true: can light schema change, false: cannot light schema change + * @throws DdlException + */ + public boolean processAddColumns(AddColumnsClause alterClause, OlapTable olapTable, + Map> indexSchemaMap, boolean ignoreSameColumn, IntSupplier colUniqueIdSupplier) + throws DdlException { List columns = alterClause.getColumns(); String targetIndexName = alterClause.getRollupName(); checkIndexExists(olapTable, targetIndexName); @@ -188,14 +215,26 @@ private void processAddColumns(AddColumnsClause alterClause, OlapTable olapTable targetIndexId = olapTable.getIndexIdByName(targetIndexName); } + //for new table calculate column unique id + if (olapTable.getMaxColUniqueId() > Column.COLUMN_UNIQUE_ID_INIT_VALUE) { + for (Column column : columns) { + column.setUniqueId(colUniqueIdSupplier.getAsInt()); + } + } + + boolean ligthSchemaChange = true; for (Column column : columns) { - addColumnInternal(olapTable, column, null, targetIndexId, baseIndexId, - indexSchemaMap, newColNameSet); + boolean result = addColumnInternal(olapTable, column, null, targetIndexId, baseIndexId, indexSchemaMap, + newColNameSet, ignoreSameColumn); + if (!result) { + ligthSchemaChange = false; + } } + return ligthSchemaChange; } - private void processDropColumn(DropColumnClause alterClause, - Table externalTable, List newSchema) throws DdlException { + private void processDropColumn(DropColumnClause alterClause, Table externalTable, List newSchema) + throws DdlException { String dropColName = alterClause.getColName(); // find column in base index and remove it @@ -208,8 +247,9 @@ private void processDropColumn(DropColumnClause alterClause, baseIter.remove(); found = true; } else { - throw new DdlException("Do not allow remove last column of table: " + externalTable.getName() - + " column: " + dropColName); + throw new DdlException( + "Do not allow remove last column of table: " + externalTable.getName() + " column: " + + dropColName); } break; } @@ -220,8 +260,23 @@ private void processDropColumn(DropColumnClause alterClause, } } - private void processDropColumn(DropColumnClause alterClause, OlapTable olapTable, + /** + * @param alterClause + * @param olapTable + * @param indexSchemaMap + * @param indexes + * @return true: can light schema change, false: cannot + * @throws DdlException + */ + private boolean processDropColumn(DropColumnClause alterClause, OlapTable olapTable, Map> indexSchemaMap, List indexes) throws DdlException { + + boolean ligthSchemaChange = false; + if (olapTable.getMaxColUniqueId() > Column.COLUMN_UNIQUE_ID_INIT_VALUE) { + //assume can light schema change. + ligthSchemaChange = true; + } + String dropColName = alterClause.getColName(); String targetIndexName = alterClause.getRollupName(); checkIndexExists(olapTable, targetIndexName); @@ -241,6 +296,7 @@ private void processDropColumn(DropColumnClause alterClause, OlapTable olapTable boolean isKey = false; for (Column column : baseSchema) { if (column.isKey() && column.getName().equalsIgnoreCase(dropColName)) { + ligthSchemaChange = false; isKey = true; break; } @@ -260,6 +316,7 @@ private void processDropColumn(DropColumnClause alterClause, OlapTable olapTable for (Column column : baseSchema) { if (column.isKey() && column.getName().equalsIgnoreCase(dropColName)) { isKey = true; + ligthSchemaChange = false; } else if (AggregateType.REPLACE == column.getAggregationType() || AggregateType.REPLACE_IF_NOT_NULL == column.getAggregationType()) { hasReplaceColumn = true; @@ -279,6 +336,7 @@ private void processDropColumn(DropColumnClause alterClause, OlapTable olapTable for (Column column : targetIndexSchema) { if (column.isKey() && column.getName().equalsIgnoreCase(dropColName)) { isKey = true; + ligthSchemaChange = false; } else if (AggregateType.REPLACE == column.getAggregationType() || AggregateType.REPLACE_IF_NOT_NULL == column.getAggregationType()) { hasReplaceColumn = true; @@ -289,6 +347,15 @@ private void processDropColumn(DropColumnClause alterClause, OlapTable olapTable "Can not drop key column when rollup has value column with REPLACE aggregation method"); } } + } else if (KeysType.DUP_KEYS == olapTable.getKeysType()) { + long baseIndexId = olapTable.getBaseIndexId(); + List baseSchema = indexSchemaMap.get(baseIndexId); + for (Column column : baseSchema) { + if (column.isKey() && column.getName().equalsIgnoreCase(dropColName)) { + ligthSchemaChange = false; + break; + } + } } Iterator it = indexes.iterator(); @@ -332,12 +399,14 @@ private void processDropColumn(DropColumnClause alterClause, OlapTable olapTable while (iter.hasNext()) { Column column = iter.next(); if (column.getName().equalsIgnoreCase(dropColName)) { + ligthSchemaChange = false; iter.remove(); break; } } } // end for index names } else { + ligthSchemaChange = false; // if specify rollup index, only drop column from specified rollup index long targetIndexId = olapTable.getIndexIdByName(targetIndexName); // find column @@ -356,11 +425,12 @@ private void processDropColumn(DropColumnClause alterClause, OlapTable olapTable throw new DdlException("Column does not exists: " + dropColName); } } + return ligthSchemaChange; } // User can modify column type and column position - private void processModifyColumn(ModifyColumnClause alterClause, - Table externalTable, List newSchema) throws DdlException { + private void processModifyColumn(ModifyColumnClause alterClause, Table externalTable, List newSchema) + throws DdlException { Column modColumn = alterClause.getColumn(); ColumnPosition columnPos = alterClause.getColPos(); @@ -427,7 +497,7 @@ private void processModifyColumn(ModifyColumnClause alterClause, // User can modify column type and column position private void processModifyColumn(ModifyColumnClause alterClause, OlapTable olapTable, - Map> indexSchemaMap) throws DdlException { + Map> indexSchemaMap) throws DdlException { Column modColumn = alterClause.getColumn(); if (KeysType.AGG_KEYS == olapTable.getKeysType()) { if (modColumn.isKey() && null != modColumn.getAggregationType()) { @@ -438,16 +508,17 @@ private void processModifyColumn(ModifyColumnClause alterClause, OlapTable olapT } } else if (KeysType.UNIQUE_KEYS == olapTable.getKeysType()) { if (null != modColumn.getAggregationType()) { - throw new DdlException("Can not assign aggregation method" - + " on column in Unique data model table: " + modColumn.getName()); + throw new DdlException("Can not assign aggregation method" + " on column in Unique data model table: " + + modColumn.getName()); } if (!modColumn.isKey()) { modColumn.setAggregationType(AggregateType.REPLACE, true); } } else { if (null != modColumn.getAggregationType()) { - throw new DdlException("Can not assign aggregation method" - + " on column in Duplicate data model table: " + modColumn.getName()); + throw new DdlException( + "Can not assign aggregation method" + " on column in Duplicate data model table: " + + modColumn.getName()); } if (!modColumn.isKey()) { modColumn.setAggregationType(AggregateType.NONE, true); @@ -518,6 +589,7 @@ private void processModifyColumn(ModifyColumnClause alterClause, OlapTable olapT Column oriColumn = schemaForFinding.get(modColIndex); // retain old column name modColumn.setName(oriColumn.getName()); + modColumn.setUniqueId(oriColumn.getUniqueId()); // handle the move operation in 'indexForFindingColumn' if has if (hasColPos) { @@ -614,8 +686,8 @@ private void processModifyColumn(ModifyColumnClause alterClause, OlapTable olapT } } - private void processReorderColumn(ReorderColumnsClause alterClause, - Table externalTable, List newSchema) throws DdlException { + private void processReorderColumn(ReorderColumnsClause alterClause, Table externalTable, List newSchema) + throws DdlException { List orderedColNames = alterClause.getColumnsByPos(); newSchema.clear(); @@ -648,7 +720,7 @@ private void processReorderColumn(ReorderColumnsClause alterClause, } private void processReorderColumn(ReorderColumnsClause alterClause, OlapTable olapTable, - Map> indexSchemaMap) throws DdlException { + Map> indexSchemaMap) throws DdlException { List orderedColNames = alterClause.getColumnsByPos(); String targetIndexName = alterClause.getRollupName(); checkIndexExists(olapTable, targetIndexName); @@ -663,7 +735,7 @@ private void processReorderColumn(ReorderColumnsClause alterClause, OlapTable ol long targetIndexId = olapTable.getIndexIdByName(targetIndexName); LinkedList newSchema = new LinkedList(); - LinkedList targetIndexSchema = indexSchemaMap.get(targetIndexId); + List targetIndexSchema = indexSchemaMap.get(targetIndexId); // check and create new ordered column list Set colNameSet = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); @@ -703,9 +775,8 @@ private void processReorderColumn(ReorderColumnsClause alterClause, OlapTable ol * Add 'newColumn' to specified index. * Modified schema will be saved in 'indexSchemaMap' */ - private void addColumnInternal(Column newColumn, ColumnPosition columnPos, - List modIndexSchema, - Set newColNameSet) throws DdlException { + private void addColumnInternal(Column newColumn, ColumnPosition columnPos, List modIndexSchema, + Set newColNameSet) throws DdlException { String newColName = newColumn.getName(); int posIndex = -1; boolean hasPos = (columnPos != null && !columnPos.isFirst()); @@ -755,16 +826,33 @@ private void addColumnInternal(Column newColumn, ColumnPosition columnPos, } } - /* - * Add 'newColumn' to specified index. - * Modified schema will be saved in 'indexSchemaMap' + /** + * @param olapTable + * @param newColumn Add 'newColumn' to specified index. + * @param columnPos + * @param targetIndexId + * @param baseIndexId + * @param indexSchemaMap Modified schema will be saved in 'indexSchemaMap' + * @param newColNameSet + * @param ignoreSameColumn + * @return true: can light schema change, false: cannot + * @throws DdlException */ - private void addColumnInternal(OlapTable olapTable, Column newColumn, ColumnPosition columnPos, - long targetIndexId, long baseIndexId, - Map> indexSchemaMap, - Set newColNameSet) throws DdlException { + private boolean addColumnInternal(OlapTable olapTable, Column newColumn, ColumnPosition columnPos, + long targetIndexId, long baseIndexId, Map> indexSchemaMap, + Set newColNameSet, boolean ignoreSameColumn) throws DdlException { + //only new table generate ColUniqueId, exist table do not. + boolean ligthSchemaChange = olapTable.getMaxColUniqueId() > Column.COLUMN_UNIQUE_ID_INIT_VALUE; String newColName = newColumn.getName(); + + //make sure olapTable has locked + if (newColumn.getUniqueId() > Integer.MAX_VALUE) { + throw new DdlException("schema change add column times overflow: " + newColName); + } + LOG.debug("table: {}, newColumn: {}, uniqueId: {}", olapTable.getName(), newColumn.getName(), + newColumn.getUniqueId()); + // check the validation of aggregation method on column. // also fill the default aggregation method if not specified. if (KeysType.AGG_KEYS == olapTable.getKeysType()) { @@ -772,30 +860,31 @@ private void addColumnInternal(OlapTable olapTable, Column newColumn, ColumnPosi throw new DdlException("Can not assign aggregation method on key column: " + newColName); } else if (null == newColumn.getAggregationType()) { newColumn.setIsKey(true); - } else if (newColumn.getAggregationType() == AggregateType.SUM - && newColumn.getDefaultValue() != null && !newColumn.getDefaultValue().equals("0")) { - throw new DdlException("The default value of '" - + newColName + "' with SUM aggregation function must be zero"); + } else if (newColumn.getAggregationType() == AggregateType.SUM && newColumn.getDefaultValue() != null + && !newColumn.getDefaultValue().equals("0")) { + throw new DdlException( + "The default value of '" + newColName + "' with SUM aggregation function must be zero"); } else if (olapTable.getDefaultDistributionInfo() instanceof RandomDistributionInfo) { if (newColumn.getAggregationType() == AggregateType.REPLACE || newColumn.getAggregationType() == AggregateType.REPLACE_IF_NOT_NULL) { - throw new DdlException("Can not add value column with aggregation type " - + newColumn.getAggregationType() + " for olap table with random distribution : " - + newColName); + throw new DdlException( + "Can not add value column with aggregation type " + newColumn.getAggregationType() + + " for olap table with random distribution : " + newColName); } } } else if (KeysType.UNIQUE_KEYS == olapTable.getKeysType()) { if (newColumn.getAggregationType() != null) { - throw new DdlException("Can not assign aggregation method" - + " on column in Unique data model table: " + newColName); + throw new DdlException( + "Can not assign aggregation method" + " on column in Unique data model table: " + newColName); } if (!newColumn.isKey()) { newColumn.setAggregationType(AggregateType.REPLACE, true); } } else { if (newColumn.getAggregationType() != null) { - throw new DdlException("Can not assign aggregation method" - + " on column in Duplicate data model table: " + newColName); + throw new DdlException( + "Can not assign aggregation method" + " on column in Duplicate data model table: " + + newColName); } if (!newColumn.isKey()) { if (targetIndexId != -1L @@ -816,13 +905,20 @@ private void addColumnInternal(OlapTable olapTable, Column newColumn, ColumnPosi throw new DdlException("BITMAP_UNION must be used in AGG_KEYS"); } + //type key column do not allow light schema change. + if (newColumn.isKey()) { + ligthSchemaChange = false; + } + // check if the new column already exist in base schema. // do not support adding new column which already exist in base schema. List baseSchema = olapTable.getBaseSchema(true); boolean found = false; + Column foundColumn = null; for (Column column : baseSchema) { if (column.getName().equalsIgnoreCase(newColName)) { found = true; + foundColumn = column; break; } } @@ -832,7 +928,11 @@ private void addColumnInternal(OlapTable olapTable, Column newColumn, ColumnPosi } else if (newColName.equalsIgnoreCase(Column.SEQUENCE_COL)) { throw new DdlException("Can not enable sequence column support, already supported sequence column."); } else { - throw new DdlException("Can not add column which already exists in base table: " + newColName); + if (ignoreSameColumn && newColumn.equals(foundColumn)) { + //for add columns rpc, allow add same type column. + } else { + throw new DdlException("Can not add column which already exists in base table: " + newColName); + } } } @@ -864,9 +964,10 @@ private void addColumnInternal(OlapTable olapTable, Column newColumn, ColumnPosi modIndexSchema = indexSchemaMap.get(baseIndexId); checkAndAddColumn(modIndexSchema, newColumn, columnPos, newColNameSet, true); if (targetIndexId == -1L) { - return; + return ligthSchemaChange; } // 2. add to rollup + ligthSchemaChange = false; modIndexSchema = indexSchemaMap.get(targetIndexId); checkAndAddColumn(modIndexSchema, newColumn, columnPos, newColNameSet, false); } @@ -876,9 +977,10 @@ private void addColumnInternal(OlapTable olapTable, Column newColumn, ColumnPosi List modIndexSchema = indexSchemaMap.get(baseIndexId); checkAndAddColumn(modIndexSchema, newColumn, columnPos, newColNameSet, true); // no specified target index. return - return; + return ligthSchemaChange; } else { // add to rollup index + ligthSchemaChange = false; List modIndexSchema = indexSchemaMap.get(targetIndexId); checkAndAddColumn(modIndexSchema, newColumn, columnPos, newColNameSet, false); @@ -902,13 +1004,15 @@ private void addColumnInternal(OlapTable olapTable, Column newColumn, ColumnPosi if (targetIndexId == -1L) { // no specified target index. return - return; + return ligthSchemaChange; } + ligthSchemaChange = false; // 2. add to rollup index modIndexSchema = indexSchemaMap.get(targetIndexId); checkAndAddColumn(modIndexSchema, newColumn, columnPos, newColNameSet, false); } + return ligthSchemaChange; } /* @@ -921,7 +1025,7 @@ private void addColumnInternal(OlapTable olapTable, Column newColumn, ColumnPosi * So that k1 will be added to base index 'twice', and we just ignore this repeat adding. */ private void checkAndAddColumn(List modIndexSchema, Column newColumn, ColumnPosition columnPos, - Set newColNameSet, boolean isBaseIndex) throws DdlException { + Set newColNameSet, boolean isBaseIndex) throws DdlException { int posIndex = -1; int lastVisibleIdx = -1; String newColName = newColumn.getName(); @@ -987,8 +1091,8 @@ private void checkAndAddColumn(List modIndexSchema, Column newColumn, Co private void checkIndexExists(OlapTable olapTable, String targetIndexName) throws DdlException { if (targetIndexName != null && !olapTable.hasMaterializedIndex(targetIndexName)) { - throw new DdlException("Index[" + targetIndexName + "] does not exist in table[" + olapTable.getName() - + "]"); + throw new DdlException( + "Index[" + targetIndexName + "] does not exist in table[" + olapTable.getName() + "]"); } } @@ -1002,7 +1106,7 @@ private void checkAssignedTargetIndexName(String baseIndexName, String targetInd } private void createJob(long dbId, OlapTable olapTable, Map> indexSchemaMap, - Map propertyMap, List indexes) throws UserException { + Map propertyMap, List indexes) throws UserException { if (olapTable.getState() == OlapTableState.ROLLUP) { throw new DdlException("Table[" + olapTable.getName() + "]'s is doing ROLLUP job"); } @@ -1022,8 +1126,8 @@ private void createJob(long dbId, OlapTable olapTable, Map { - if (!alterJobsV2.isDone() && !activeSchemaChangeJobsV2.containsKey(alterJobsV2.getJobId()) - && activeSchemaChangeJobsV2.size() < MAX_ACTIVE_SCHEMA_CHANGE_JOB_V2_SIZE) { - if (FeConstants.runningUnitTest) { - alterJobsV2.run(); - } else { - schemaChangeThreadPool.submit(() -> { - if (activeSchemaChangeJobsV2.putIfAbsent(alterJobsV2.getJobId(), alterJobsV2) == null) { - try { - alterJobsV2.run(); - } finally { - activeSchemaChangeJobsV2.remove(alterJobsV2.getJobId()); - } - } - }); + runnableSchemaChangeJobV2.values().forEach(alterJobsV2 -> { + if (!alterJobsV2.isDone() && !activeSchemaChangeJobsV2.containsKey(alterJobsV2.getJobId()) + && activeSchemaChangeJobsV2.size() < MAX_ACTIVE_SCHEMA_CHANGE_JOB_V2_SIZE) { + if (FeConstants.runningUnitTest) { + alterJobsV2.run(); + } else { + schemaChangeThreadPool.submit(() -> { + if (activeSchemaChangeJobsV2.putIfAbsent(alterJobsV2.getJobId(), alterJobsV2) == null) { + try { + alterJobsV2.run(); + } finally { + activeSchemaChangeJobsV2.remove(alterJobsV2.getJobId()); + } } - } - }); + }); + } + } + }); } @Override @@ -1465,8 +1571,8 @@ private void getAlterJobV2Infos(Database db, List alterJobsV2, continue; } if (ctx != null) { - if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv( - ctx, db.getFullName(), alterJob.getTableName(), PrivPredicate.ALTER)) { + if (!Catalog.getCurrentCatalog().getAuth() + .checkTblPriv(ctx, db.getFullName(), alterJob.getTableName(), PrivPredicate.ALTER)) { continue; } } @@ -1483,6 +1589,19 @@ public void process(List alterClauses, String clusterName, Database throws UserException { olapTable.writeLockOrDdlException(); try { + //alterClauses can or cannot light schema change + boolean ligthSchemaChange = true; + //for multi add colmuns clauses + IntSupplier colUniqueIdSupplier = new IntSupplier() { + public int pendingMaxColUniqueId = olapTable.getMaxColUniqueId(); + + @Override + public int getAsInt() { + pendingMaxColUniqueId++; + return pendingMaxColUniqueId; + } + }; + // index id -> index schema Map> indexSchemaMap = new HashMap<>(); for (Map.Entry> entry : olapTable.getIndexIdToSchema(true).entrySet()) { @@ -1509,8 +1628,8 @@ public void process(List alterClauses, String clusterName, Database } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_DISTRIBUTION_TYPE)) { String distributionType = properties.get(PropertyAnalyzer.PROPERTIES_DISTRIBUTION_TYPE); if (!distributionType.equalsIgnoreCase("random")) { - throw new DdlException("Only support modifying distribution type of table from" - + " hash to random"); + throw new DdlException( + "Only support modifying distribution type of table from" + " hash to random"); } Catalog.getCurrentCatalog().convertDistributionType(db, olapTable); return; @@ -1523,14 +1642,13 @@ public void process(List alterClauses, String clusterName, Database } else if (DynamicPartitionUtil.checkDynamicPartitionPropertiesExist(properties)) { if (!olapTable.dynamicPartitionExists()) { try { - DynamicPartitionUtil.checkInputDynamicPartitionProperties( - properties, olapTable.getPartitionInfo()); + DynamicPartitionUtil.checkInputDynamicPartitionProperties(properties, + olapTable.getPartitionInfo()); } catch (DdlException e) { // This table is not a dynamic partition table // and didn't supply all dynamic partition properties - throw new DdlException("Table " + db.getFullName() + "." - + olapTable.getName() + " is not a dynamic partition table." - + " Use command `HELP ALTER TABLE` " + throw new DdlException("Table " + db.getFullName() + "." + olapTable.getName() + + " is not a dynamic partition table." + " Use command `HELP ALTER TABLE` " + "to see how to change a normal table to a dynamic partition table."); } } @@ -1538,8 +1656,8 @@ public void process(List alterClauses, String clusterName, Database return; } else if (properties.containsKey( "default." + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION)) { - Preconditions.checkNotNull(properties.get("default." - + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION)); + Preconditions.checkNotNull( + properties.get("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION)); Catalog.getCurrentCatalog().modifyTableDefaultReplicaAllocation(db, olapTable, properties); return; } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION)) { @@ -1559,36 +1677,63 @@ public void process(List alterClauses, String clusterName, Database if (alterClause instanceof AddColumnClause) { // add column - processAddColumn((AddColumnClause) alterClause, olapTable, indexSchemaMap); + boolean clauseCanLigthSchemaChange = processAddColumn((AddColumnClause) alterClause, olapTable, + indexSchemaMap, colUniqueIdSupplier); + if (clauseCanLigthSchemaChange == false) { + ligthSchemaChange = false; + } } else if (alterClause instanceof AddColumnsClause) { // add columns - processAddColumns((AddColumnsClause) alterClause, olapTable, indexSchemaMap); + boolean clauseCanLigthSchemaChange = processAddColumns((AddColumnsClause) alterClause, olapTable, + indexSchemaMap, false, colUniqueIdSupplier); + if (clauseCanLigthSchemaChange == false) { + ligthSchemaChange = false; + } } else if (alterClause instanceof DropColumnClause) { // drop column and drop indexes on this column - processDropColumn((DropColumnClause) alterClause, olapTable, indexSchemaMap, newIndexes); + boolean clauseCanLigthSchemaChange = processDropColumn((DropColumnClause) alterClause, olapTable, + indexSchemaMap, newIndexes); + if (clauseCanLigthSchemaChange == false) { + ligthSchemaChange = false; + } } else if (alterClause instanceof ModifyColumnClause) { // modify column processModifyColumn((ModifyColumnClause) alterClause, olapTable, indexSchemaMap); + ligthSchemaChange = false; } else if (alterClause instanceof ReorderColumnsClause) { // reorder column processReorderColumn((ReorderColumnsClause) alterClause, olapTable, indexSchemaMap); + ligthSchemaChange = false; } else if (alterClause instanceof ModifyTablePropertiesClause) { // modify table properties // do nothing, properties are already in propertyMap + ligthSchemaChange = false; } else if (alterClause instanceof CreateIndexClause) { if (processAddIndex((CreateIndexClause) alterClause, olapTable, newIndexes)) { return; } + ligthSchemaChange = false; } else if (alterClause instanceof DropIndexClause) { if (processDropIndex((DropIndexClause) alterClause, olapTable, newIndexes)) { return; } + ligthSchemaChange = false; } else { Preconditions.checkState(false); } } // end for alter clauses - createJob(db.getId(), olapTable, indexSchemaMap, propertyMap, newIndexes); + LOG.debug("processAddColumns, table: {}({}), ligthSchemaChange: {}", olapTable.getName(), olapTable.getId(), + ligthSchemaChange); + + if (ligthSchemaChange) { + long jobId = Catalog.getCurrentCatalog().getNextId(); + //for schema change add/drop value column optimize, direct modify table meta. + modifyTableAddOrDropColumns(db, olapTable, indexSchemaMap, newIndexes, jobId, false); + return; + } else { + createJob(db.getId(), olapTable, indexSchemaMap, propertyMap, newIndexes); + } } finally { olapTable.writeUnlock(); } @@ -1690,9 +1835,8 @@ public void updateTableInMemoryMeta(Database db, String tableName, Map partitionNames, Map properties) - throws DdlException, MetaNotFoundException { + public void updatePartitionsInMemoryMeta(Database db, String tableName, List partitionNames, + Map properties) throws DdlException, MetaNotFoundException { OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableName, Table.TableType.OLAP); boolean isInMemory = Boolean.parseBoolean(properties.get(PropertyAnalyzer.PROPERTIES_INMEMORY)); if (isInMemory == olapTable.isInMemory()) { @@ -1714,10 +1858,8 @@ public void updatePartitionsInMemoryMeta(Database db, * Update one specified partition's in-memory property by partition name of table * This operation may return partial successfully, with a exception to inform user to retry */ - public void updatePartitionInMemoryMeta(Database db, - String tableName, - String partitionName, - boolean isInMemory) throws UserException { + public void updatePartitionInMemoryMeta(Database db, String tableName, String partitionName, boolean isInMemory) + throws UserException { // be id -> Map>> beIdToTabletIdWithHash = Maps.newHashMap(); OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableName, Table.TableType.OLAP); @@ -1733,8 +1875,8 @@ public void updatePartitionInMemoryMeta(Database db, int schemaHash = olapTable.getSchemaHashByIndexId(index.getId()); for (Tablet tablet : index.getTablets()) { for (Replica replica : tablet.getReplicas()) { - Set> tabletIdWithHash = - beIdToTabletIdWithHash.computeIfAbsent(replica.getBackendId(), k -> Sets.newHashSet()); + Set> tabletIdWithHash = beIdToTabletIdWithHash.computeIfAbsent( + replica.getBackendId(), k -> Sets.newHashSet()); tabletIdWithHash.add(new Pair<>(tablet.getId(), schemaHash)); } } @@ -1748,16 +1890,16 @@ public void updatePartitionInMemoryMeta(Database db, AgentBatchTask batchTask = new AgentBatchTask(); for (Map.Entry>> kv : beIdToTabletIdWithHash.entrySet()) { countDownLatch.addMark(kv.getKey(), kv.getValue()); - UpdateTabletMetaInfoTask task = new UpdateTabletMetaInfoTask(kv.getKey(), kv.getValue(), - isInMemory, countDownLatch); + UpdateTabletMetaInfoTask task = new UpdateTabletMetaInfoTask(kv.getKey(), kv.getValue(), isInMemory, + countDownLatch); batchTask.addTask(task); } if (!FeConstants.runningUnitTest) { // send all tasks and wait them finished AgentTaskQueue.addBatchTask(batchTask); AgentTaskExecutor.submit(batchTask); - LOG.info("send update tablet meta task for table {}, partitions {}, number: {}", - tableName, partitionName, batchTask.getTaskNum()); + LOG.info("send update tablet meta task for table {}, partitions {}, number: {}", tableName, partitionName, + batchTask.getTaskNum()); // estimate timeout long timeout = Config.tablet_create_timeout_second * 1000L * totalTaskNum; @@ -1779,8 +1921,8 @@ public void updatePartitionInMemoryMeta(Database db, } else { List>>> unfinishedMarks = countDownLatch.getLeftMarks(); // only show at most 3 results - List>>> subList - = unfinishedMarks.subList(0, Math.min(unfinishedMarks.size(), 3)); + List>>> subList = unfinishedMarks.subList(0, + Math.min(unfinishedMarks.size(), 3)); if (!subList.isEmpty()) { errMsg += " Unfinished mark: " + Joiner.on(", ").join(subList); } @@ -1815,11 +1957,11 @@ public void cancel(CancelStmt stmt) throws DdlException { // find from new alter jobs first List schemaChangeJobV2List = getUnfinishedAlterJobV2ByTableId(olapTable.getId()); // current schemaChangeJob job doesn't support batch operation,so just need to get one job - schemaChangeJobV2 = schemaChangeJobV2List.size() == 0 - ? null : Iterables.getOnlyElement(schemaChangeJobV2List); + schemaChangeJobV2 = schemaChangeJobV2List.size() == 0 ? null + : Iterables.getOnlyElement(schemaChangeJobV2List); if (schemaChangeJobV2 == null) { - throw new DdlException("Table[" + tableName + "] is under schema change state" - + " but could not find related job"); + throw new DdlException( + "Table[" + tableName + "] is under schema change state" + " but could not find related job"); } } finally { olapTable.writeUnlock(); @@ -1851,8 +1993,8 @@ private boolean processAddIndex(CreateIndexClause alterClause, OlapTable olapTab for (Index existedIdx : existedIndexes) { if (existedIdx.getIndexName().equalsIgnoreCase(indexDef.getIndexName())) { if (indexDef.isSetIfNotExists()) { - LOG.info("create index[{}] which already exists on table[{}]", - indexDef.getIndexName(), olapTable.getName()); + LOG.info("create index[{}] which already exists on table[{}]", indexDef.getIndexName(), + olapTable.getName()); return true; } throw new DdlException("index `" + indexDef.getIndexName() + "` already exist."); @@ -1882,8 +2024,8 @@ private boolean processAddIndex(CreateIndexClause alterClause, OlapTable olapTab * Returns true if the index does not exist, there is no need to create the job to drop the index. * Otherwise return false, there is need to create a job to drop the index. */ - private boolean processDropIndex(DropIndexClause alterClause, OlapTable olapTable, - List indexes) throws DdlException { + private boolean processDropIndex(DropIndexClause alterClause, OlapTable olapTable, List indexes) + throws DdlException { String indexName = alterClause.getIndexName(); List existedIndexes = olapTable.getIndexes(); Index found = null; @@ -1943,4 +2085,172 @@ public void replayAlterJobV2(AlterJobV2 alterJob) { } super.replayAlterJobV2(alterJob); } + + // the invoker should keep table's write lock + public void modifyTableAddOrDropColumns(Database db, OlapTable olapTable, + Map> indexSchemaMap, + List indexes, long jobId, boolean isReplay) throws DdlException { + + LOG.debug("indexSchemaMap:{}, indexes:{}", indexSchemaMap, indexes); + if (olapTable.getState() == OlapTableState.ROLLUP) { + throw new DdlException("Table[" + olapTable.getName() + "]'s is doing ROLLUP job"); + } + + // for now table's state can only be NORMAL + Preconditions.checkState(olapTable.getState() == OlapTableState.NORMAL, olapTable.getState().name()); + + // for bitmapIndex + boolean hasIndexChange = false; + Set newSet = new HashSet<>(indexes); + Set oriSet = new HashSet<>(olapTable.getIndexes()); + if (!newSet.equals(oriSet)) { + hasIndexChange = true; + } + + // begin checking each table + // ATTN: DO NOT change any meta in this loop + Map> changedIndexIdToSchema = Maps.newHashMap(); + for (Long alterIndexId : indexSchemaMap.keySet()) { + // Must get all columns including invisible columns. + // Because in alter process, all columns must be considered. + List alterSchema = indexSchemaMap.get(alterIndexId); + + LOG.debug("index[{}] is changed. start checking...", alterIndexId); + // 1. check order: a) has key; b) value after key + boolean meetValue = false; + boolean hasKey = false; + for (Column column : alterSchema) { + if (column.isKey() && meetValue) { + throw new DdlException( + "Invalid column order. value should be after key. index[" + olapTable.getIndexNameById( + alterIndexId) + "]"); + } + if (!column.isKey()) { + meetValue = true; + } else { + hasKey = true; + } + } + if (!hasKey) { + throw new DdlException("No key column left. index[" + olapTable.getIndexNameById(alterIndexId) + "]"); + } + + // 2. check partition key + PartitionInfo partitionInfo = olapTable.getPartitionInfo(); + if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) { + List partitionColumns = partitionInfo.getPartitionColumns(); + for (Column partitionCol : partitionColumns) { + boolean found = false; + for (Column alterColumn : alterSchema) { + if (alterColumn.nameEquals(partitionCol.getName(), true)) { + found = true; + break; + } + } // end for alterColumns + + if (!found && alterIndexId == olapTable.getBaseIndexId()) { + // 2.1 partition column cannot be deleted. + throw new DdlException( + "Partition column[" + partitionCol.getName() + "] cannot be dropped. index[" + + olapTable.getIndexNameById(alterIndexId) + "]"); + } + } // end for partitionColumns + } + + // 3. check distribution key: + DistributionInfo distributionInfo = olapTable.getDefaultDistributionInfo(); + if (distributionInfo.getType() == DistributionInfoType.HASH) { + List distributionColumns = ((HashDistributionInfo) distributionInfo).getDistributionColumns(); + for (Column distributionCol : distributionColumns) { + boolean found = false; + for (Column alterColumn : alterSchema) { + if (alterColumn.nameEquals(distributionCol.getName(), true)) { + found = true; + break; + } + } // end for alterColumns + if (!found && alterIndexId == olapTable.getBaseIndexId()) { + // 2.2 distribution column cannot be deleted. + throw new DdlException( + "Distribution column[" + distributionCol.getName() + "] cannot be dropped. index[" + + olapTable.getIndexNameById(alterIndexId) + "]"); + } + } // end for distributionCols + } + + // 5. store the changed columns for edit log + changedIndexIdToSchema.put(alterIndexId, alterSchema); + + LOG.debug("schema change[{}-{}-{}] check pass.", db.getId(), olapTable.getId(), alterIndexId); + } // end for indices + + if (changedIndexIdToSchema.isEmpty() && !hasIndexChange) { + throw new DdlException("Nothing is changed. please check your alter stmt."); + } + + //update base index schema + long baseIndexId = olapTable.getBaseIndexId(); + List indexIds = new ArrayList(); + indexIds.add(baseIndexId); + indexIds.addAll(olapTable.getIndexIdListExceptBaseIndex()); + for (int i = 0; i < indexIds.size(); i++) { + List indexSchema = indexSchemaMap.get(indexIds.get(i)); + MaterializedIndexMeta currentIndexMeta = olapTable.getIndexMetaByIndexId(indexIds.get(i)); + currentIndexMeta.setSchema(indexSchema); + + int currentSchemaVersion = currentIndexMeta.getSchemaVersion(); + int newSchemaVersion = currentSchemaVersion + 1; + currentIndexMeta.setSchemaVersion(newSchemaVersion); + } + olapTable.setIndexes(indexes); + olapTable.rebuildFullSchema(); + + //update max column unique id + int maxColUniqueId = olapTable.getMaxColUniqueId(); + for (Column column : indexSchemaMap.get(olapTable.getBaseIndexId())) { + if (column.getUniqueId() > maxColUniqueId) { + maxColUniqueId = column.getUniqueId(); + } + } + olapTable.setMaxColUniqueId(maxColUniqueId); + + if (!isReplay) { + TableAddOrDropColumnsInfo info = new TableAddOrDropColumnsInfo(db.getId(), olapTable.getId(), + indexSchemaMap, indexes, jobId); + LOG.debug("logModifyTableAddOrDropColumns info:{}", info); + Catalog.getCurrentCatalog().getEditLog().logModifyTableAddOrDropColumns(info); + } + + //for compatibility, we need create a finished state schema change job v2 + + SchemaChangeJobV2 schemaChangeJob = new SchemaChangeJobV2(jobId, db.getId(), olapTable.getId(), + olapTable.getName(), 1000); + schemaChangeJob.setJobState(AlterJobV2.JobState.FINISHED); + schemaChangeJob.setFinishedTimeMs(System.currentTimeMillis()); + this.addAlterJobV2(schemaChangeJob); + + LOG.info("finished modify table's add or drop columns. table: {}, is replay: {}", olapTable.getName(), + isReplay); + } + + public void replayModifyTableAddOrDropColumns(TableAddOrDropColumnsInfo info) throws MetaNotFoundException { + LOG.debug("info:{}", info); + long dbId = info.getDbId(); + long tableId = info.getTableId(); + Map> indexSchemaMap = info.getIndexSchemaMap(); + List indexes = info.getIndexes(); + long jobId = info.getJobId(); + + Database db = Catalog.getCurrentCatalog().getInternalDataSource().getDbOrMetaException(dbId); + OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId, TableType.OLAP); + olapTable.writeLock(); + try { + modifyTableAddOrDropColumns(db, olapTable, indexSchemaMap, indexes, jobId, true); + } catch (DdlException e) { + // should not happen + LOG.warn("failed to replay modify table add or drop columns", e); + } finally { + olapTable.writeUnlock(); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java index d7a0662bb2c6e89..6dcb35ad26a0d27 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java @@ -437,7 +437,7 @@ protected void runWaitingTxnJob() throws AlterCancelException { long originIdxId = indexIdMap.get(shadowIdxId); int shadowSchemaHash = indexSchemaVersionAndHashMap.get(shadowIdxId).schemaHash; int originSchemaHash = tbl.getSchemaHashByIndexId(indexIdMap.get(shadowIdxId)); - + List originSchemaColumns = tbl.getSchemaByIndexId(originIdxId); for (Tablet shadowTablet : shadowIdx.getTablets()) { long shadowTabletId = shadowTablet.getId(); long originTabletId = partitionIndexTabletMap.get(partitionId, shadowIdxId).get(shadowTabletId); @@ -446,7 +446,7 @@ protected void runWaitingTxnJob() throws AlterCancelException { AlterReplicaTask rollupTask = new AlterReplicaTask(shadowReplica.getBackendId(), dbId, tableId, partitionId, shadowIdxId, originIdxId, shadowTabletId, originTabletId, shadowReplica.getId(), shadowSchemaHash, originSchemaHash, visibleVersion, jobId, - JobType.SCHEMA_CHANGE, defineExprs, descTable); + JobType.SCHEMA_CHANGE, defineExprs, descTable, originSchemaColumns); schemaChangeBatchTask.addTask(rollupTask); } } @@ -627,6 +627,16 @@ private void onFinished(OlapTable tbl) { tbl.setStorageFormat(storageFormat); } + //update max column unique id + int maxColUniqueId = tbl.getMaxColUniqueId(); + for (Column column : tbl.getFullSchema()) { + if (column.getUniqueId() > maxColUniqueId) { + maxColUniqueId = column.getUniqueId(); + } + } + tbl.setMaxColUniqueId(maxColUniqueId); + LOG.debug("fullSchema:{}, maxColUniqueId:{}", tbl.getFullSchema(), maxColUniqueId); + tbl.setState(OlapTableState.NORMAL); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java index b1b471c603d14e4..75b3290030263f2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java @@ -400,7 +400,7 @@ public String toSql() { public Column toColumn() { return new Column(name, typeDef.getType(), isKey, aggregateType, isAllowNull, defaultValue.value, comment, - visible, defaultValue.defaultValueExprDef); + visible, defaultValue.defaultValueExprDef, Column.COLUMN_UNIQUE_ID_INIT_VALUE); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SlotDescriptor.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SlotDescriptor.java index 4f041dfe01d0b53..8c405011aeb0c30 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SlotDescriptor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SlotDescriptor.java @@ -29,11 +29,14 @@ import com.google.common.base.MoreObjects; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.Collections; import java.util.List; public class SlotDescriptor { + private static final Logger LOG = LogManager.getLogger(SlotDescriptor.class); private final SlotId id; private final TupleDescriptor parent; private Type type; @@ -278,27 +281,26 @@ public boolean layoutEquals(SlotDescriptor other) { // TODO public TSlotDescriptor toThrift() { - if (originType != null) { - return new TSlotDescriptor(id.asInt(), parent.getId().asInt(), originType.toThrift(), -1, - byteOffset, nullIndicatorByte, - nullIndicatorBit, ((column != null) ? column.getName() : ""), slotIdx, isMaterialized); - } else { - return new TSlotDescriptor(id.asInt(), parent.getId().asInt(), type.toThrift(), -1, - byteOffset, nullIndicatorByte, - nullIndicatorBit, ((column != null) ? column.getName() : ""), slotIdx, isMaterialized); + + TSlotDescriptor tSlotDescriptor = new TSlotDescriptor(id.asInt(), parent.getId().asInt(), + (originType != null ? originType.toThrift() : type.toThrift()), -1, byteOffset, nullIndicatorByte, + nullIndicatorBit, ((column != null) ? column.getName() : ""), slotIdx, isMaterialized); + + if (column != null) { + LOG.debug("column name:{}, column unique id:{}", column.getName(), column.getUniqueId()); + tSlotDescriptor.setColUniqueId(column.getUniqueId()); } + return tSlotDescriptor; } public String debugString() { String colStr = (column == null ? "null" : column.getName()); String typeStr = (type == null ? "null" : type.toString()); String parentTupleId = (parent == null) ? "null" : parent.getId().toString(); - return MoreObjects.toStringHelper(this).add("id", id.asInt()).add("parent", parentTupleId) - .add("col", colStr).add("type", typeStr).add("materialized", isMaterialized) - .add("byteSize", byteSize).add("byteOffset", byteOffset) - .add("nullIndicatorByte", nullIndicatorByte) - .add("nullIndicatorBit", nullIndicatorBit) - .add("slotIdx", slotIdx).toString(); + return MoreObjects.toStringHelper(this).add("id", id.asInt()).add("parent", parentTupleId).add("col", colStr) + .add("type", typeStr).add("materialized", isMaterialized).add("byteSize", byteSize) + .add("byteOffset", byteOffset).add("nullIndicatorByte", nullIndicatorByte) + .add("nullIndicatorBit", nullIndicatorBit).add("slotIdx", slotIdx).toString(); } @Override @@ -311,10 +313,8 @@ public String getExplainString(String prefix) { String colStr = (column == null ? "null" : column.getName()); String typeStr = (type == null ? "null" : type.toString()); String parentTupleId = (parent == null) ? "null" : parent.getId().toString(); - builder.append(prefix).append("SlotDescriptor{") - .append("id=").append(id) - .append(", col=").append(colStr) - .append(", type=").append(typeStr).append("}\n"); + builder.append(prefix).append("SlotDescriptor{").append("id=").append(id).append(", col=").append(colStr) + .append(", type=").append(typeStr).append("}\n"); prefix += " "; builder.append(prefix).append("parent=").append(parentTupleId).append("\n"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SlotRef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SlotRef.java index 6a0976404b4b0d4..0bd5470ed0c183b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SlotRef.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SlotRef.java @@ -71,7 +71,7 @@ public SlotRef(TableName tblName, String col) { public SlotRef(SlotDescriptor desc) { super(); this.tblName = null; - this.col = null; + this.col = desc.getColumn() != null ? desc.getColumn().getName() : null; this.desc = desc; this.type = desc.getType(); // TODO(zc): label is meaningful diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index 4020ffa72f07e63..f2246c96dae1a37 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -71,6 +71,7 @@ import org.apache.doris.analysis.RestoreStmt; import org.apache.doris.analysis.RollupRenameClause; import org.apache.doris.analysis.ShowAlterStmt.AlterType; +import org.apache.doris.analysis.TableName; import org.apache.doris.analysis.TableRenameClause; import org.apache.doris.analysis.TruncateTableStmt; import org.apache.doris.analysis.UninstallPluginStmt; @@ -251,6 +252,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Optional; import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; @@ -567,21 +569,19 @@ private Catalog(boolean isCheckpointCatalog) { this.metaContext.setThreadLocalInfo(); this.stat = new TabletSchedulerStat(); - this.tabletScheduler = new TabletScheduler(this, systemInfo, - tabletInvertedIndex, stat, Config.tablet_rebalancer_type); + this.tabletScheduler = new TabletScheduler(this, systemInfo, tabletInvertedIndex, stat, + Config.tablet_rebalancer_type); this.tabletChecker = new TabletChecker(this, systemInfo, tabletScheduler, stat); // The pendingLoadTaskScheduler's queue size should not less than Config.desired_max_waiting_jobs. // So that we can guarantee that all submitted load jobs can be scheduled without being starved. this.pendingLoadTaskScheduler = new MasterTaskExecutor("pending_load_task_scheduler", - Config.async_pending_load_task_pool_size, - Config.desired_max_waiting_jobs, !isCheckpointCatalog); + Config.async_pending_load_task_pool_size, Config.desired_max_waiting_jobs, !isCheckpointCatalog); // The loadingLoadTaskScheduler's queue size is unlimited, so that it can receive all loading tasks // created after pending tasks finish. And don't worry about the high concurrency, because the // concurrency is limited by Config.desired_max_waiting_jobs and Config.async_loading_load_task_pool_size. this.loadingLoadTaskScheduler = new MasterTaskExecutor("loading_load_task_scheduler", - Config.async_loading_load_task_pool_size, - Integer.MAX_VALUE, !isCheckpointCatalog); + Config.async_loading_load_task_pool_size, Integer.MAX_VALUE, !isCheckpointCatalog); this.loadJobScheduler = new LoadJobScheduler(); this.loadManager = new LoadManager(loadJobScheduler); @@ -792,8 +792,8 @@ public void initialize(String[] args) throws Exception { // 1. check and create dirs and files File meta = new File(metaDir); if (!meta.exists()) { - LOG.warn("Doris' meta dir {} does not exist." - + " You need to create it before starting FE", meta.getAbsolutePath()); + LOG.warn("Doris' meta dir {} does not exist." + " You need to create it before starting FE", + meta.getAbsolutePath()); throw new Exception(meta.getAbsolutePath() + " does not exist, will exit"); } @@ -866,8 +866,7 @@ private void getClusterIdAndRole() throws IOException { } // check file integrity, if has. - if ((roleFile.exists() && !versionFile.exists()) - || (!roleFile.exists() && versionFile.exists())) { + if ((roleFile.exists() && !versionFile.exists()) || (!roleFile.exists() && versionFile.exists())) { throw new IOException("role file and version file must both exist or both not exist. " + "please specific one helper node to recover. will exit."); } @@ -914,12 +913,12 @@ private void getClusterIdAndRole() throws IOException { // But is metadata_failure_recovery is true, // we will not check it because this may be a FE migration. String[] split = nodeName.split("_"); - if (Config.metadata_failure_recovery.equals("false") - && !selfNode.first.equalsIgnoreCase(split[0])) { - throw new IOException("the self host " + selfNode.first - + " does not equal to the host in ROLE" - + " file " + split[0] + ". You need to set 'priority_networks' config" - + " in fe.conf to match the host " + split[0]); + if (Config.metadata_failure_recovery.equals("false") && !selfNode.first.equalsIgnoreCase( + split[0])) { + throw new IOException( + "the self host " + selfNode.first + " does not equal to the host in ROLE" + " file " + + split[0] + ". You need to set 'priority_networks' config" + + " in fe.conf to match the host " + split[0]); } } } @@ -988,8 +987,8 @@ private void getClusterIdAndRole() throws IOException { if (!versionFile.exists()) { // If the version file doesn't exist, download it from helper node if (!getVersionFileFromHelper(rightHelperNode)) { - throw new IOException("fail to download version file from " - + rightHelperNode.first + " will exit."); + throw new IOException( + "fail to download version file from " + rightHelperNode.first + " will exit."); } // NOTE: cluster_id will be init when Storage object is constructed, @@ -1028,8 +1027,8 @@ private void getClusterIdAndRole() throws IOException { Preconditions.checkNotNull(token); Preconditions.checkNotNull(remoteToken); if (!token.equals(remoteToken)) { - throw new IOException("token is not equal with helper node " - + rightHelperNode.first + ". will exit."); + throw new IOException( + "token is not equal with helper node " + rightHelperNode.first + ". will exit."); } } } catch (Exception e) { @@ -1051,8 +1050,7 @@ private void getClusterIdAndRole() throws IOException { } Preconditions.checkState(helperNodes.size() == 1); - LOG.info("finished to get cluster id: {}, role: {} and node name: {}", - clusterId, role.name(), nodeName); + LOG.info("finished to get cluster id: {}, role: {} and node name: {}", clusterId, role.name(), nodeName); } public static String genFeNodeName(String host, int port, boolean isOldStyle) { @@ -1072,13 +1070,13 @@ private boolean getFeNodeTypeAndNameFromHelpers() { Pair rightHelperNode = null; for (Pair helperNode : helperNodes) { try { - URL url = new URL("http://" + helperNode.first + ":" + Config.http_port - + "/role?host=" + selfNode.first + "&port=" + selfNode.second); + URL url = new URL("http://" + helperNode.first + ":" + Config.http_port + "/role?host=" + selfNode.first + + "&port=" + selfNode.second); HttpURLConnection conn = null; conn = (HttpURLConnection) url.openConnection(); if (conn.getResponseCode() != 200) { - LOG.warn("failed to get fe node type from helper node: {}. response code: {}", - helperNode, conn.getResponseCode()); + LOG.warn("failed to get fe node type from helper node: {}. response code: {}", helperNode, + conn.getResponseCode()); continue; } @@ -1166,9 +1164,8 @@ private void getHelperNodes(String[] args) throws Exception { * In this case, some errors have caused users to be troubled. * So here directly exit the program and inform the user to avoid unnecessary trouble. */ - throw new AnalysisException( - "Do not specify the helper node to FE itself. " - + "Please specify it to the existing running Master or Follower FE"); + throw new AnalysisException("Do not specify the helper node to FE itself. " + + "Please specify it to the existing running Master or Follower FE"); } helperNodes.add(helperHostPort); } @@ -1188,8 +1185,7 @@ private void getHelperNodeFromDeployManager() throws Exception { // 1. check if this is the first time to start up File roleFile = new File(this.imageDir, Storage.ROLE_FILE); File versionFile = new File(this.imageDir, Storage.VERSION_FILE); - if ((roleFile.exists() && !versionFile.exists()) - || (!roleFile.exists() && versionFile.exists())) { + if ((roleFile.exists() && !versionFile.exists()) || (!roleFile.exists() && versionFile.exists())) { throw new Exception("role file and version file must both exist or both not exist. " + "please specific one helper node to recover. will exit."); } @@ -1459,7 +1455,8 @@ private void checkLowerCaseTableNames() { if (Config.lower_case_table_names != GlobalVariable.lowerCaseTableNames) { LOG.error("The configuration of \'lower_case_table_names\' does not support modification, " + "the expected value is {}, but the actual value is {}", - GlobalVariable.lowerCaseTableNames, Config.lower_case_table_names); + GlobalVariable.lowerCaseTableNames, + Config.lower_case_table_names); System.exit(-1); } LOG.info("lower_case_table_names is {}", GlobalVariable.lowerCaseTableNames); @@ -1513,8 +1510,7 @@ private void getNewImage(Pair helperNode) throws IOException { StorageInfo info = getStorageInfo(infoUrl); long version = info.getImageSeq(); if (version > localImageVersion) { - String url = "http://" + helperNode.first + ":" + Config.http_port - + "/image?version=" + version; + String url = "http://" + helperNode.first + ":" + Config.http_port + "/image?version=" + version; String filename = Storage.IMAGE + "." + version; File dir = new File(this.imageDir); MetaHelper.getRemoteFile(url, HTTP_TIMEOUT_SECOND * 1000, MetaHelper.getOutputStream(filename, dir)); @@ -1554,8 +1550,8 @@ private StorageInfo getStorageInfo(URL url) throws IOException { connection.setReadTimeout(HTTP_TIMEOUT_SECOND * 1000); String response; - try (BufferedReader bufferedReader - = new BufferedReader(new InputStreamReader(connection.getInputStream()))) { + try (BufferedReader bufferedReader = new BufferedReader( + new InputStreamReader(connection.getInputStream()))) { String line; StringBuilder sb = new StringBuilder(); while ((line = bufferedReader.readLine()) != null) { @@ -2030,8 +2026,8 @@ public long saveLoadJob(CountingDataOutputStream dos, long checksum) throws IOEx public long saveExportJob(CountingDataOutputStream dos, long checksum) throws IOException { long curTime = System.currentTimeMillis(); - List jobs = exportMgr.getJobs().stream() - .filter(t -> !t.isExpired(curTime)).collect(Collectors.toList()); + List jobs = exportMgr.getJobs().stream().filter(t -> !t.isExpired(curTime)) + .collect(Collectors.toList()); int size = jobs.size(); checksum ^= size; dos.writeInt(size); @@ -2617,11 +2613,11 @@ public void replayRenameDatabase(String dbName, String newDbName) { * 4. set table id and base index id * 5. set bloom filter columns * 6. set and build TableProperty includes: - * 6.1. dynamicProperty - * 6.2. replicationNum - * 6.3. inMemory - * 6.4. storageFormat - * 6.5. compressionType + * 6.1. dynamicProperty + * 6.2. replicationNum + * 6.3. inMemory + * 6.4. storageFormat + * 6.5. compressionType * 7. set index meta * 8. check colocation properties * 9. create tablet in BE @@ -2692,10 +2688,8 @@ public static void getDdlStmt(DdlStmt ddlStmt, String dbName, TableIf table, Lis // 1.2 other table type sb.append("CREATE "); - if (table.getType() == TableType.ODBC - || table.getType() == TableType.MYSQL - || table.getType() == TableType.ELASTICSEARCH - || table.getType() == TableType.BROKER + if (table.getType() == TableType.ODBC || table.getType() == TableType.MYSQL + || table.getType() == TableType.ELASTICSEARCH || table.getType() == TableType.BROKER || table.getType() == TableType.HIVE) { sb.append("EXTERNAL "); } @@ -2834,8 +2828,7 @@ public static void getDdlStmt(DdlStmt ddlStmt, String dbName, TableIf table, Lis Preconditions.checkState(partitionId.size() == 1); partition = olapTable.getPartition(partitionId.get(0)); } - sb.append(partition.getVisibleVersion()) - .append("\""); + sb.append(partition.getVisibleVersion()).append("\""); } // colocateTable @@ -2897,8 +2890,8 @@ public static void getDdlStmt(DdlStmt ddlStmt, String dbName, TableIf table, Lis sb.append("\"password\" = \"").append(hidePassword ? "" : mysqlTable.getPasswd()).append("\",\n"); sb.append("\"charset\" = \"").append(mysqlTable.getCharset()).append("\",\n"); } else { - sb.append("\"odbc_catalog_resource\" = \"") - .append(mysqlTable.getOdbcCatalogResourceName()).append("\",\n"); + sb.append("\"odbc_catalog_resource\" = \"").append(mysqlTable.getOdbcCatalogResourceName()) + .append("\",\n"); } sb.append("\"database\" = \"").append(mysqlTable.getMysqlDatabaseName()).append("\",\n"); sb.append("\"table\" = \"").append(mysqlTable.getMysqlTableName()).append("\"\n"); @@ -2918,8 +2911,8 @@ public static void getDdlStmt(DdlStmt ddlStmt, String dbName, TableIf table, Lis sb.append("\"driver\" = \"").append(odbcTable.getOdbcDriver()).append("\",\n"); sb.append("\"odbc_type\" = \"").append(odbcTable.getOdbcTableTypeName()).append("\",\n"); } else { - sb.append("\"odbc_catalog_resource\" = \"") - .append(odbcTable.getOdbcCatalogResourceName()).append("\",\n"); + sb.append("\"odbc_catalog_resource\" = \"").append(odbcTable.getOdbcCatalogResourceName()) + .append("\",\n"); } sb.append("\"database\" = \"").append(odbcTable.getOdbcDatabaseName()).append("\",\n"); sb.append("\"table\" = \"").append(odbcTable.getOdbcTableName()).append("\"\n"); @@ -3038,8 +3031,7 @@ public static void getDdlStmt(DdlStmt ddlStmt, String dbName, TableIf table, Lis sb.append(")"); } sb.append("(\"version_info\" = \""); - sb.append(partition.getVisibleVersion()) - .append("\""); + sb.append(partition.getVisibleVersion()).append("\""); sb.append(");"); addPartitionStmt.add(sb.toString()); } @@ -3160,8 +3152,7 @@ public void replayDropFrontend(Frontend frontend) { LOG.error(frontend.toString() + " does not exist."); return; } - if (removedFe.getRole() == FrontendNodeType.FOLLOWER - || removedFe.getRole() == FrontendNodeType.REPLICA) { + if (removedFe.getRole() == FrontendNodeType.FOLLOWER || removedFe.getRole() == FrontendNodeType.REPLICA) { helperNodes.remove(Pair.create(removedFe.getHost(), removedFe.getEditLogPort())); } @@ -3217,8 +3208,8 @@ public HashMap getPartitionIdToStorageMediumMap() { for (Partition partition : olapTable.getAllPartitions()) { long partitionId = partition.getId(); DataProperty dataProperty = partitionInfo.getDataProperty(partition.getId()); - Preconditions.checkNotNull(dataProperty, partition.getName() - + ", pId:" + partitionId + ", db: " + dbId + ", tbl: " + tableId); + Preconditions.checkNotNull(dataProperty, + partition.getName() + ", pId:" + partitionId + ", db: " + dbId + ", tbl: " + tableId); if (dataProperty.getStorageMedium() == TStorageMedium.SSD && dataProperty.getCooldownTimeMs() < currentTimeMs) { // expire. change to HDD. @@ -3257,8 +3248,8 @@ public HashMap getPartitionIdToStorageMediumMap() { // use try lock to avoid blocking a long time. // if block too long, backend report rpc will timeout. if (!olapTable.tryWriteLockIfExist(Table.TRY_LOCK_TIMEOUT_MS, TimeUnit.MILLISECONDS)) { - LOG.warn("try get table {} writelock but failed" - + " when checking backend storage medium", table.getName()); + LOG.warn("try get table {} writelock but failed" + " when checking backend storage medium", + table.getName()); continue; } Preconditions.checkState(olapTable.isWriteLockHeldByCurrentThread()); @@ -3278,16 +3269,13 @@ public HashMap getPartitionIdToStorageMediumMap() { DataProperty hddProperty = new DataProperty(TStorageMedium.HDD); partitionInfo.setDataProperty(partition.getId(), hddProperty); storageMediumMap.put(partitionId, TStorageMedium.HDD); - LOG.debug("partition[{}-{}-{}] storage medium changed from SSD to HDD", - dbId, tableId, partitionId); + LOG.debug("partition[{}-{}-{}] storage medium changed from SSD to HDD", dbId, tableId, + partitionId); // log - ModifyPartitionInfo info = - new ModifyPartitionInfo(db.getId(), olapTable.getId(), - partition.getId(), - hddProperty, - ReplicaAllocation.NOT_SET, - partitionInfo.getIsInMemory(partition.getId())); + ModifyPartitionInfo info = new ModifyPartitionInfo(db.getId(), olapTable.getId(), + partition.getId(), hddProperty, ReplicaAllocation.NOT_SET, + partitionInfo.getIsInMemory(partition.getId())); editLog.logModifyPartition(info); } } // end for partitions @@ -3721,15 +3709,15 @@ public void modifyTableColocate(Database db, OlapTable table, String colocateGro if (bucketsNum == -1) { bucketsNum = partition.getDistributionInfo().getBucketNum(); } else if (bucketsNum != partition.getDistributionInfo().getBucketNum()) { - throw new DdlException("Partitions in table " + table.getName() - + " have different buckets number"); + throw new DdlException( + "Partitions in table " + table.getName() + " have different buckets number"); } if (replicaAlloc == null) { replicaAlloc = partitionInfo.getReplicaAllocation(partition.getId()); } else if (!replicaAlloc.equals(partitionInfo.getReplicaAllocation(partition.getId()))) { - throw new DdlException("Partitions in table " + table.getName() - + " have different replica allocation."); + throw new DdlException( + "Partitions in table " + table.getName() + " have different replica allocation."); } } } @@ -3778,8 +3766,7 @@ public void modifyTableColocate(Database db, OlapTable table, String colocateGro TablePropertyInfo info = new TablePropertyInfo(table.getId(), groupId, properties); editLog.logModifyTableColocate(info); } - LOG.info("finished modify table's colocation property. table: {}, is replay: {}", - table.getName(), isReplay); + LOG.info("finished modify table's colocation property. table: {}, is replay: {}", table.getName(), isReplay); } public void replayModifyTableColocate(TablePropertyInfo info) throws MetaNotFoundException { @@ -3871,8 +3858,8 @@ public void renamePartition(Database db, OlapTable table, PartitionRenameClause if (table.getPartitionInfo().getType() != PartitionType.RANGE && table.getPartitionInfo().getType() != PartitionType.LIST) { - throw new DdlException("Table[" + table.getName() + "] is single partitioned. " - + "no need to rename partition name."); + throw new DdlException( + "Table[" + table.getName() + "] is single partitioned. " + "no need to rename partition name."); } String partitionName = renameClause.getPartitionName(); @@ -3940,17 +3927,17 @@ public void modifyTableDynamicPartition(Database db, OlapTable table, Map origDynamicProperties = tableProperty.getOriginDynamicPartitionProperty(); origDynamicProperties.putAll(properties); - Map analyzedDynamicPartition = DynamicPartitionUtil - .analyzeDynamicPartition(origDynamicProperties, table.getPartitionInfo()); + Map analyzedDynamicPartition = DynamicPartitionUtil.analyzeDynamicPartition( + origDynamicProperties, table.getPartitionInfo()); tableProperty.modifyTableProperties(analyzedDynamicPartition); tableProperty.buildDynamicProperty(); } DynamicPartitionUtil.registerOrRemoveDynamicPartitionTable(db.getId(), table, false); - dynamicPartitionScheduler.createOrUpdateRuntimeInfo( - table.getId(), DynamicPartitionScheduler.LAST_UPDATE_TIME, TimeUtils.getCurrentFormatTime()); - ModifyTablePropertyOperationLog info - = new ModifyTablePropertyOperationLog(db.getId(), table.getId(), logProperties); + dynamicPartitionScheduler.createOrUpdateRuntimeInfo(table.getId(), DynamicPartitionScheduler.LAST_UPDATE_TIME, + TimeUtils.getCurrentFormatTime()); + ModifyTablePropertyOperationLog info = new ModifyTablePropertyOperationLog(db.getId(), table.getId(), + logProperties); editLog.logDynamicPartition(info); } @@ -3964,22 +3951,24 @@ private void convertDynamicPartitionReplicaNumToReplicaAllocation(Map properties) throws UserException { + public void modifyTableReplicaAllocation(Database db, OlapTable table, Map properties) + throws UserException { Preconditions.checkArgument(table.isWriteLockHeldByCurrentThread()); String defaultReplicationNumName = "default." + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM; PartitionInfo partitionInfo = table.getPartitionInfo(); if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) { - throw new DdlException("This is a partitioned table, you should specify partitions" - + " with MODIFY PARTITION clause." - + " If you want to set default replication number, please use '" + defaultReplicationNumName - + "' instead of '" + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM + "' to escape misleading."); + throw new DdlException( + "This is a partitioned table, you should specify partitions" + " with MODIFY PARTITION clause." + + " If you want to set default replication number, please use '" + defaultReplicationNumName + + "' instead of '" + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM + + "' to escape misleading."); } String partitionName = table.getName(); Partition partition = table.getPartition(partitionName); @@ -3997,13 +3986,14 @@ public void modifyTableReplicaAllocation(Database db, OlapTable table, ModifyPartitionInfo info = new ModifyPartitionInfo(db.getId(), table.getId(), partition.getId(), newDataProperty, replicaAlloc, isInMemory); editLog.logModifyPartition(info); - LOG.debug("modify partition[{}-{}-{}] replica allocation to {}", db.getId(), table.getId(), - partition.getName(), replicaAlloc.toCreateStmt()); + LOG.debug("modify partition[{}-{}-{}] replica allocation to {}", db.getId(), table.getId(), partition.getName(), + replicaAlloc.toCreateStmt()); } /** * Set default replication allocation for a specified table. * You can see the default replication allocation by executing Show Create Table stmt. + * * @param db * @param table * @param properties @@ -4022,8 +4012,8 @@ public void modifyTableDefaultReplicaAllocation(Database db, OlapTable table, Ma tableProperty.buildReplicaAllocation(); // log - ModifyTablePropertyOperationLog info - = new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); + ModifyTablePropertyOperationLog info = new ModifyTablePropertyOperationLog(db.getId(), table.getId(), + properties); editLog.logModifyReplicationNum(info); LOG.debug("modify table[{}] replication num to {}", table.getName(), properties.get(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)); @@ -4045,13 +4035,13 @@ public void modifyTableInMemoryMeta(Database db, OlapTable table, Map properties = info.getProperties(); @@ -4099,17 +4089,18 @@ public void modifyDefaultDistributionBucketNum(Database db, OlapTable olapTable, DistributionInfo distributionInfo = distributionDesc.toDistributionInfo(baseSchema); // for now. we only support modify distribution's bucket num if (distributionInfo.getType() != defaultDistributionInfo.getType()) { - throw new DdlException("Cannot change distribution type when modify" - + " default distribution bucket num"); + throw new DdlException( + "Cannot change distribution type when modify" + " default distribution bucket num"); } if (distributionInfo.getType() == DistributionInfoType.HASH) { HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distributionInfo; List newDistriCols = hashDistributionInfo.getDistributionColumns(); - List defaultDistriCols = ((HashDistributionInfo) defaultDistributionInfo) - .getDistributionColumns(); + List defaultDistriCols + = ((HashDistributionInfo) defaultDistributionInfo).getDistributionColumns(); if (!newDistriCols.equals(defaultDistriCols)) { - throw new DdlException("Cannot assign hash distribution with different distribution cols. " - + "default is: " + defaultDistriCols); + throw new DdlException( + "Cannot assign hash distribution with different distribution cols. " + "default is: " + + defaultDistriCols); } } @@ -4163,8 +4154,8 @@ public void cancelAlterCluster(CancelAlterSystemStmt stmt) throws DdlException { // Switch catalog of this sesseion. public void changeCatalog(ConnectContext ctx, String catalogName) throws DdlException { if (dataSourceMgr.getCatalogNullable(catalogName) == null) { - throw new DdlException(ErrorCode.ERR_UNKNOWN_CATALOG.formatErrorMsg( - catalogName), ErrorCode.ERR_UNKNOWN_CATALOG); + throw new DdlException(ErrorCode.ERR_UNKNOWN_CATALOG.formatErrorMsg(catalogName), + ErrorCode.ERR_UNKNOWN_CATALOG); } ctx.changeDefaultCatalog(catalogName); } @@ -4596,8 +4587,8 @@ public void replayBackendTabletsInfo(BackendTabletsInfo backendTabletsInfo) { Replica replica = tablet.getReplicaById(info.getReplicaId()); if (replica != null) { replica.setBad(true); - LOG.debug("get replica {} of tablet {} on backend {} to bad when replaying", - info.getReplicaId(), info.getTabletId(), info.getBackendId()); + LOG.debug("get replica {} of tablet {} on backend {} to bad when replaying", info.getReplicaId(), + info.getTabletId(), info.getBackendId()); } } finally { olapTable.writeUnlock(); @@ -4674,8 +4665,8 @@ public void replaceTempPartition(Database db, OlapTable olapTable, ReplacePartit ReplacePartitionOperationLog info = new ReplacePartitionOperationLog(db.getId(), olapTable.getId(), partitionNames, tempPartitionNames, isStrictRange, useTempPartitionName); editLog.logReplaceTempPartition(info); - LOG.info("finished to replace partitions {} with temp partitions {} from table: {}", - clause.getPartitionNames(), clause.getTempPartitionNames(), olapTable.getName()); + LOG.info("finished to replace partitions {} with temp partitions {} from table: {}", clause.getPartitionNames(), + clause.getTempPartitionNames(), olapTable.getName()); } public void replayReplaceTempPartition(ReplacePartitionOperationLog replaceTempPartitionLog) @@ -4687,8 +4678,7 @@ public void replayReplaceTempPartition(ReplacePartitionOperationLog replaceTempP olapTable.writeLock(); try { olapTable.replaceTempPartitions(replaceTempPartitionLog.getPartitions(), - replaceTempPartitionLog.getTempPartitions(), - replaceTempPartitionLog.isStrictRange(), + replaceTempPartitionLog.getTempPartitions(), replaceTempPartitionLog.isStrictRange(), replaceTempPartitionLog.useTempPartitionName()); } catch (DdlException e) { throw new MetaNotFoundException(e); @@ -4764,8 +4754,8 @@ public void replaySetReplicaStatus(SetReplicaStatusOperationLog log) throws Meta setReplicaStatusInternal(log.getTabletId(), log.getBackendId(), log.getReplicaStatus(), true); } - private void setReplicaStatusInternal(long tabletId, long backendId, - ReplicaStatus status, boolean isReplay) throws MetaNotFoundException { + private void setReplicaStatusInternal(long tabletId, long backendId, ReplicaStatus status, boolean isReplay) + throws MetaNotFoundException { try { TabletMeta meta = tabletInvertedIndex.getTabletMeta(tabletId); if (meta == null) { @@ -4782,12 +4772,12 @@ private void setReplicaStatusInternal(long tabletId, long backendId, if (status == ReplicaStatus.BAD || status == ReplicaStatus.OK) { if (replica.setBad(status == ReplicaStatus.BAD)) { if (!isReplay) { - SetReplicaStatusOperationLog log - = new SetReplicaStatusOperationLog(backendId, tabletId, status); + SetReplicaStatusOperationLog log = new SetReplicaStatusOperationLog(backendId, tabletId, + status); getEditLog().logSetReplicaStatus(log); } - LOG.info("set replica {} of tablet {} on backend {} as {}. is replay: {}", - replica.getId(), tabletId, backendId, status, isReplay); + LOG.info("set replica {} of tablet {} on backend {} as {}. is replay: {}", replica.getId(), + tabletId, backendId, status, isReplay); } } } finally { @@ -4944,4 +4934,15 @@ public int getFollowerCount() { } return count; } + + public TableName getTableNameByTableId(Long tableId) { + for (String dbName : getInternalDataSource().getDbNames()) { + DatabaseIf db = getInternalDataSource().getDbNullable(dbName); + Optional table = db.getTable(tableId); + if (table.isPresent()) { + return new TableName(db.getFullName(), table.get().getName()); + } + } + return null; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java index 3badbcab4bc758d..25cd47f62839a12 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java @@ -20,6 +20,7 @@ import org.apache.doris.alter.SchemaChangeHandler; import org.apache.doris.analysis.DefaultValueExprDef; import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.IndexDef; import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.StringLiteral; import org.apache.doris.common.AnalysisException; @@ -55,6 +56,7 @@ public class Column implements Writable { public static final String DELETE_SIGN = "__DORIS_DELETE_SIGN__"; public static final String SEQUENCE_COL = "__DORIS_SEQUENCE_COL__"; private static final String COLUMN_ARRAY_CHILDREN = "item"; + public static final int COLUMN_UNIQUE_ID_INIT_VALUE = -1; @SerializedName(value = "name") private String name; @@ -93,6 +95,9 @@ public class Column implements Writable { @SerializedName(value = "defaultValueExprDef") private DefaultValueExprDef defaultValueExprDef; // used for default value + @SerializedName(value = "uniqueId") + private int uniqueId; + public Column() { this.name = ""; this.type = Type.NULL; @@ -102,6 +107,7 @@ public Column() { this.visible = true; this.defineExpr = null; this.children = new ArrayList<>(Type.MAX_NESTING_DEPTH); + this.uniqueId = -1; } public Column(String name, PrimitiveType dataType) { @@ -123,11 +129,13 @@ public Column(String name, Type type, boolean isKey, AggregateType aggregateType public Column(String name, Type type, boolean isKey, AggregateType aggregateType, boolean isAllowNull, String defaultValue, String comment) { - this(name, type, isKey, aggregateType, isAllowNull, defaultValue, comment, true, null); + this(name, type, isKey, aggregateType, isAllowNull, defaultValue, comment, true, null, + COLUMN_UNIQUE_ID_INIT_VALUE); } public Column(String name, Type type, boolean isKey, AggregateType aggregateType, boolean isAllowNull, - String defaultValue, String comment, boolean visible, DefaultValueExprDef defaultValueExprDef) { + String defaultValue, String comment, boolean visible, DefaultValueExprDef defaultValueExprDef, + int colUniqueId) { this.name = name; if (this.name == null) { this.name = ""; @@ -149,6 +157,7 @@ public Column(String name, Type type, boolean isKey, AggregateType aggregateType this.visible = visible; this.children = new ArrayList<>(Type.MAX_NESTING_DEPTH); createChildrenColumn(this.type, this); + this.uniqueId = colUniqueId; } public Column(Column column) { @@ -164,6 +173,7 @@ public Column(Column column) { this.stats = column.getStats(); this.visible = column.visible; this.children = column.getChildren(); + this.uniqueId = column.getUniqueId(); } public void createChildrenColumn(Type type, Column column) { @@ -366,6 +376,7 @@ public TColumn toThrift() { tColumn.setVisible(visible); toChildrenThrift(this, tColumn); + tColumn.setColUniqueId(uniqueId); // ATTN: // Currently, this `toThrift()` method is only used from CreateReplicaTask. // And CreateReplicaTask does not need `defineExpr` field. @@ -688,4 +699,23 @@ public String getSignatureString(Map typeStringMap) { sb.append(defaultValue == null ? "" : defaultValue); return sb.toString(); } + + public void setUniqueId(int colUniqueId) { + this.uniqueId = colUniqueId; + } + + public int getUniqueId() { + return this.uniqueId; + } + + public void setIndexFlag(TColumn tColumn, List indexes) { + for (Index index : indexes) { + if (index.getIndexType() == IndexDef.IndexType.BITMAP) { + List columns = index.getColumns(); + if (tColumn.getColumnName().equals(columns.get(0))) { + tColumn.setHasBitmapIndex(true); + } + } + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java index ede7c32a178a168..8e600c599159863 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java @@ -104,6 +104,14 @@ public List getSchema(boolean full) { } } + public void setSchema(List newSchema) { + this.schema = newSchema; + } + + public void setSchemaHash(int newSchemaHash) { + this.schemaHash = newSchemaHash; + } + public int getSchemaHash() { return schemaHash; } @@ -112,6 +120,10 @@ public short getShortKeyColumnCount() { return shortKeyColumnCount; } + public void setSchemaVersion(int newSchemaVersion) { + this.schemaVersion = newSchemaVersion; + } + public int getSchemaVersion() { return schemaVersion; } 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 80ab60064d8b75b..9e23ebe3749cf48 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 @@ -38,6 +38,7 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; import org.apache.doris.common.FeConstants; +import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.io.DeepCopy; @@ -141,6 +142,8 @@ public enum OlapTableState { private TableProperty tableProperty; + private int maxColUniqueId = Column.COLUMN_UNIQUE_ID_INIT_VALUE; + public OlapTable() { // for persist super(TableType.OLAP); @@ -191,6 +194,20 @@ public TableProperty getTableProperty() { return this.tableProperty; } + //take care: only use at create olap table. + public int incAndGetMaxColUniqueId() { + this.maxColUniqueId++; + return this.maxColUniqueId; + } + + public int getMaxColUniqueId() { + return this.maxColUniqueId; + } + + public void setMaxColUniqueId(int maxColUniqueId) { + this.maxColUniqueId = maxColUniqueId; + } + public boolean dynamicPartitionExists() { return tableProperty != null && tableProperty.getDynamicPartitionProperty() != null @@ -1144,6 +1161,7 @@ public void write(DataOutput out) throws IOException { } tempPartitions.write(out); + out.writeInt(maxColUniqueId); } @Override @@ -1235,6 +1253,10 @@ public void readFields(DataInput in) throws IOException { } } tempPartitions.unsetPartitionInfo(); + + if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_112) { + maxColUniqueId = in.readInt(); + } // In the present, the fullSchema could be rebuilt by schema change while the properties is changed by MV. // After that, some properties of fullSchema and nameToColumn may be not same as properties of base columns. // So, here we need to rebuild the fullSchema to ensure the correctness of the properties. diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java index 124c4532618bc85..887ae74c59bccc3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java @@ -81,7 +81,7 @@ public class Table extends MetaObject implements Writable, TableIf { * Schema change (c3 to bigint) * When OlapTable is changing schema, the fullSchema is (c1 int, c2 int, c3 int, SHADOW_NAME_PRFIX_c3 bigint) * The fullSchema of OlapTable is mainly used by Scanner of Load job. - * + *

* If you want to get the mv columns, you should call getIndexToSchema in Subclass OlapTable. */ protected List fullSchema; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalDataSource.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalDataSource.java index 90797a0d66261e4..048451a5255bf03 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalDataSource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalDataSource.java @@ -339,8 +339,8 @@ public void recreateTabletInvertIndex() { Collection allPartitions = olapTable.getAllPartitions(); for (Partition partition : allPartitions) { long partitionId = partition.getId(); - TStorageMedium medium = - olapTable.getPartitionInfo().getDataProperty(partitionId).getStorageMedium(); + TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty(partitionId) + .getStorageMedium(); for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { long indexId = index.getId(); int schemaHash = olapTable.getSchemaHashByIndexId(indexId); @@ -474,8 +474,8 @@ public void dropDb(DropDbStmt stmt) throws DdlException { fullNameToDb.remove(db.getAttachDb()); db.setDbState(DbState.NORMAL); info.setUpdateDbState(DbState.NORMAL); - final Cluster cluster = - nameToCluster.get(ClusterNamespace.getClusterNameFromFullName(db.getAttachDb())); + final Cluster cluster = nameToCluster.get( + ClusterNamespace.getClusterNameFromFullName(db.getAttachDb())); final BaseParam param = new BaseParam(); param.addStringParam(db.getAttachDb()); param.addLongParam(db.getId()); @@ -844,8 +844,7 @@ public void dropTable(DropTableStmt stmt) throws DdlException { OlapTable olapTable = (OlapTable) table; if ((olapTable.getState() != OlapTableState.NORMAL)) { throw new DdlException("The table [" + tableName + "]'s state is " + olapTable.getState() - + ", cannot be dropped." - + " please cancel the operation on olap table firstly." + + ", cannot be dropped." + " please cancel the operation on olap table firstly." + " If you want to forcibly drop(cannot be recovered)," + " please use \"DROP table FORCE\"."); } @@ -924,10 +923,9 @@ private void unprotectAddReplica(OlapTable olapTable, ReplicaPersistInfo info) { schemaHash = olapTable.getSchemaHashByIndexId(info.getIndexId()); } - Replica replica = - new Replica(info.getReplicaId(), info.getBackendId(), info.getVersion(), schemaHash, info.getDataSize(), - info.getRowCount(), ReplicaState.NORMAL, info.getLastFailedVersion(), - info.getLastSuccessVersion()); + Replica replica = new Replica(info.getReplicaId(), info.getBackendId(), info.getVersion(), schemaHash, + info.getDataSize(), info.getRowCount(), ReplicaState.NORMAL, info.getLastFailedVersion(), + info.getLastSuccessVersion()); tablet.addReplica(replica); } @@ -1090,8 +1088,8 @@ public void createTableLike(CreateTableLikeStmt stmt) throws DdlException { } finally { table.readUnlock(); } - CreateTableStmt parsedCreateTableStmt = - (CreateTableStmt) SqlParserUtils.parseAndAnalyzeStmt(createTableStmt.get(0), ConnectContext.get()); + CreateTableStmt parsedCreateTableStmt = (CreateTableStmt) SqlParserUtils.parseAndAnalyzeStmt( + createTableStmt.get(0), ConnectContext.get()); parsedCreateTableStmt.setTableName(stmt.getTableName()); parsedCreateTableStmt.setIfNotExists(stmt.isIfNotExists()); createTable(parsedCreateTableStmt); @@ -1171,8 +1169,8 @@ public void replayCreateTable(String dbName, Table table) throws MetaNotFoundExc long tableId = table.getId(); for (Partition partition : olapTable.getAllPartitions()) { long partitionId = partition.getId(); - TStorageMedium medium = - olapTable.getPartitionInfo().getDataProperty(partitionId).getStorageMedium(); + TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty(partitionId) + .getStorageMedium(); for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.ALL)) { long indexId = mIndex.getId(); int schemaHash = olapTable.getSchemaHashByIndexId(indexId); @@ -1256,8 +1254,8 @@ public void addPartition(Database db, String tableName, AddPartitionClause addPa if (distributionInfo.getType() == DistributionInfoType.HASH) { HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distributionInfo; List newDistriCols = hashDistributionInfo.getDistributionColumns(); - List defaultDistriCols = - ((HashDistributionInfo) defaultDistributionInfo).getDistributionColumns(); + List defaultDistriCols + = ((HashDistributionInfo) defaultDistributionInfo).getDistributionColumns(); if (!newDistriCols.equals(defaultDistriCols)) { throw new DdlException( "Cannot assign hash distribution with different distribution cols. " + "default is: " @@ -1430,9 +1428,8 @@ public void replayAddPartition(PartitionPersistInfo info) throws MetaNotFoundExc for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { long indexId = index.getId(); int schemaHash = olapTable.getSchemaHashByIndexId(indexId); - TabletMeta tabletMeta = - new TabletMeta(info.getDbId(), info.getTableId(), partition.getId(), index.getId(), - schemaHash, info.getDataProperty().getStorageMedium()); + TabletMeta tabletMeta = new TabletMeta(info.getDbId(), info.getTableId(), partition.getId(), + index.getId(), schemaHash, info.getDataProperty().getStorageMedium()); for (Tablet tablet : index.getTablets()) { long tabletId = tablet.getId(); invertedIndex.addTablet(tabletId, tabletMeta); @@ -1592,11 +1589,10 @@ private Partition createPartitionWithIndices(String clusterName, long dbId, long long backendId = replica.getBackendId(); long replicaId = replica.getId(); countDownLatch.addMark(backendId, tabletId); - CreateReplicaTask task = - new CreateReplicaTask(backendId, dbId, tableId, partitionId, indexId, tabletId, replicaId, - shortKeyColumnCount, schemaHash, version, keysType, storageType, storageMedium, - schema, bfColumns, bfFpp, countDownLatch, indexes, isInMemory, tabletType, - dataSortInfo, compressionType); + CreateReplicaTask task = new CreateReplicaTask(backendId, dbId, tableId, partitionId, indexId, + tabletId, replicaId, shortKeyColumnCount, schemaHash, version, keysType, storageType, + storageMedium, schema, bfColumns, bfFpp, countDownLatch, indexes, isInMemory, tabletType, + dataSortInfo, compressionType); task.setStorageFormat(storageFormat); batchTask.addTask(task); // add to AgentTaskQueue for handling finish report. @@ -1693,9 +1689,15 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep // create table long tableId = Catalog.getCurrentCatalog().getNextId(); - OlapTable olapTable = - new OlapTable(tableId, tableName, baseSchema, keysType, partitionInfo, defaultDistributionInfo, - indexes); + OlapTable olapTable = new OlapTable(tableId, tableName, baseSchema, keysType, partitionInfo, + defaultDistributionInfo, indexes); + + for (Column column : baseSchema) { + column.setUniqueId(olapTable.incAndGetMaxColUniqueId()); + LOG.debug("table: {}, newColumn: {}, uniqueId: {}", olapTable.getName(), column.getName(), + column.getUniqueId()); + } + olapTable.setComment(stmt.getComment()); // set base index id @@ -1725,8 +1727,8 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep olapTable.setCompressionType(compressionType); // check data sort properties - DataSortInfo dataSortInfo = - PropertyAnalyzer.analyzeDataSortInfo(properties, keysType, keysDesc.keysColumnSize(), storageFormat); + DataSortInfo dataSortInfo = PropertyAnalyzer.analyzeDataSortInfo(properties, keysType, + keysDesc.keysColumnSize(), storageFormat); olapTable.setDataSortInfo(dataSortInfo); // analyze bloom filter columns @@ -1758,8 +1760,8 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep olapTable.setReplicationAllocation(replicaAlloc); // set in memory - boolean isInMemory = - PropertyAnalyzer.analyzeBooleanProp(properties, PropertyAnalyzer.PROPERTIES_INMEMORY, false); + boolean isInMemory = PropertyAnalyzer.analyzeBooleanProp(properties, PropertyAnalyzer.PROPERTIES_INMEMORY, + false); olapTable.setIsInMemory(isInMemory); // set remote storage @@ -1782,8 +1784,8 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep long partitionId = partitionNameToId.get(tableName); DataProperty dataProperty = null; try { - dataProperty = - PropertyAnalyzer.analyzeDataProperty(stmt.getProperties(), DataProperty.DEFAULT_DATA_PROPERTY); + dataProperty = PropertyAnalyzer.analyzeDataProperty(stmt.getProperties(), + DataProperty.DEFAULT_DATA_PROPERTY); } catch (AnalysisException e) { throw new DdlException(e.getMessage()); } @@ -1851,8 +1853,8 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep // set rollup index meta to olap table List rollupColumns = Catalog.getCurrentCatalog().getMaterializedViewHandler() .checkAndPrepareMaterializedView(addRollupClause, olapTable, baseRollupIndex, false); - short rollupShortKeyColumnCount = - Catalog.calcShortKeyColumnCount(rollupColumns, alterClause.getProperties()); + short rollupShortKeyColumnCount = Catalog.calcShortKeyColumnCount(rollupColumns, + alterClause.getProperties()); int rollupSchemaHash = Util.generateSchemaHash(); long rollupIndexId = Catalog.getCurrentCatalog().getNextId(); olapTable.setIndexMeta(rollupIndexId, addRollupClause.getRollupName(), rollupColumns, schemaVersion, @@ -1984,10 +1986,10 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep // we have added these index to memory, only need to persist here if (Catalog.getCurrentColocateIndex().isColocateTable(tableId)) { GroupId groupId = Catalog.getCurrentColocateIndex().getGroup(tableId); - Map>> backendsPerBucketSeq = - Catalog.getCurrentColocateIndex().getBackendsPerBucketSeq(groupId); - ColocatePersistInfo info = - ColocatePersistInfo.createForAddTable(groupId, tableId, backendsPerBucketSeq); + Map>> backendsPerBucketSeq = Catalog.getCurrentColocateIndex() + .getBackendsPerBucketSeq(groupId); + ColocatePersistInfo info = ColocatePersistInfo.createForAddTable(groupId, tableId, + backendsPerBucketSeq); Catalog.getCurrentCatalog().getEditLog().logColocateAddTable(info); } LOG.info("successfully create table[{};{}]", tableName, tableId); @@ -2104,8 +2106,8 @@ private void createHiveTable(Database db, CreateTableStmt stmt) throws DdlExcept HiveTable hiveTable = new HiveTable(tableId, tableName, columns, stmt.getProperties()); hiveTable.setComment(stmt.getComment()); // check hive table whether exists in hive database - HiveMetaStoreClient hiveMetaStoreClient = - HiveMetaStoreClientHelper.getClient(hiveTable.getHiveProperties().get(HiveTable.HIVE_METASTORE_URIS)); + HiveMetaStoreClient hiveMetaStoreClient = HiveMetaStoreClientHelper.getClient( + hiveTable.getHiveProperties().get(HiveTable.HIVE_METASTORE_URIS)); if (!HiveMetaStoreClientHelper.tableExists(hiveMetaStoreClient, hiveTable.getHiveDb(), hiveTable.getHiveTable())) { throw new DdlException(String.format("Table [%s] dose not exist in Hive.", hiveTable.getHiveDbTable())); @@ -2133,9 +2135,8 @@ private void createHudiTable(Database db, CreateTableStmt stmt) throws DdlExcept throw new DdlException( String.format("Table [%s] dose not exist in Hive Metastore.", hudiTable.getHmsTableIdentifer())); } - org.apache.hadoop.hive.metastore.api.Table hiveTable = - HiveMetaStoreClientHelper.getTable(hudiTable.getHmsDatabaseName(), hudiTable.getHmsTableName(), - metastoreUris); + org.apache.hadoop.hive.metastore.api.Table hiveTable = HiveMetaStoreClientHelper.getTable( + hudiTable.getHmsDatabaseName(), hudiTable.getHmsTableName(), metastoreUris); if (!HudiUtils.isHudiTable(hiveTable)) { throw new DdlException(String.format("Table [%s] is not a hudi table.", hudiTable.getHmsTableIdentifer())); } @@ -2222,8 +2223,8 @@ private void createTablets(String clusterName, MaterializedIndex index, ReplicaS for (List backendIds : chosenBackendIds.values()) { for (long backendId : backendIds) { long replicaId = Catalog.getCurrentCatalog().getNextId(); - Replica replica = - new Replica(replicaId, backendId, replicaState, version, tabletMeta.getOldSchemaHash()); + Replica replica = new Replica(replicaId, backendId, replicaState, version, + tabletMeta.getOldSchemaHash()); tablet.addReplica(replica); totalReplicaNum++; } @@ -2396,8 +2397,8 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti truncateTableInternal(olapTable, newPartitions, truncateEntireTable); // write edit log - TruncateTableInfo info = - new TruncateTableInfo(db.getId(), olapTable.getId(), newPartitions, truncateEntireTable); + TruncateTableInfo info = new TruncateTableInfo(db.getId(), olapTable.getId(), newPartitions, + truncateEntireTable); Catalog.getCurrentCatalog().getEditLog().logTruncateTable(info); } finally { olapTable.writeUnlock(); @@ -2442,13 +2443,13 @@ public void replayTruncateTable(TruncateTableInfo info) throws MetaNotFoundExcep TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); for (Partition partition : info.getPartitions()) { long partitionId = partition.getId(); - TStorageMedium medium = - olapTable.getPartitionInfo().getDataProperty(partitionId).getStorageMedium(); + TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty(partitionId) + .getStorageMedium(); for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.ALL)) { long indexId = mIndex.getId(); int schemaHash = olapTable.getSchemaHashByIndexId(indexId); - TabletMeta tabletMeta = - new TabletMeta(db.getId(), olapTable.getId(), partitionId, indexId, schemaHash, medium); + TabletMeta tabletMeta = new TabletMeta(db.getId(), olapTable.getId(), partitionId, indexId, + schemaHash, medium); for (Tablet tablet : mIndex.getTablets()) { long tabletId = tablet.getId(); invertedIndex.addTablet(tabletId, tabletMeta); @@ -2501,8 +2502,8 @@ public void createCluster(CreateClusterStmt stmt) throws DdlException { if (nameToCluster.containsKey(clusterName)) { ErrorReport.reportDdlException(ErrorCode.ERR_CLUSTER_HAS_EXIST, clusterName); } else { - List backendList = - Catalog.getCurrentSystemInfo().createCluster(clusterName, stmt.getInstanceNum()); + List backendList = Catalog.getCurrentSystemInfo() + .createCluster(clusterName, stmt.getInstanceNum()); // 1: BE returned is less than requested, throws DdlException. // 2: BE returned is more than or equal to 0, succeeds. if (backendList != null || stmt.getInstanceNum() == 0) { @@ -2960,8 +2961,8 @@ public Set getMigrations() { if (db.getDbState() == DbState.MOVE) { int tabletTotal = 0; int tabletQuorum = 0; - final Set beIds = - Sets.newHashSet(Catalog.getCurrentSystemInfo().getClusterBackendIds(db.getClusterName())); + final Set beIds = Sets.newHashSet( + Catalog.getCurrentSystemInfo().getClusterBackendIds(db.getClusterName())); final Set tableNames = db.getTableNamesWithLock(); for (String tableName : tableNames) { @@ -2974,8 +2975,8 @@ public Set getMigrations() { olapTable.readLock(); try { for (Partition partition : olapTable.getPartitions()) { - ReplicaAllocation replicaAlloc = - olapTable.getPartitionInfo().getReplicaAllocation(partition.getId()); + ReplicaAllocation replicaAlloc = olapTable.getPartitionInfo() + .getReplicaAllocation(partition.getId()); short totalReplicaNum = replicaAlloc.getTotalReplicaNum(); for (MaterializedIndex materializedIndex : partition.getMaterializedIndices( IndexExtState.ALL)) { @@ -3064,8 +3065,8 @@ public long loadCluster(DataInputStream dis, long checksum) throws IOException, public void initDefaultCluster() { final List backendList = Lists.newArrayList(); - final List defaultClusterBackends = - Catalog.getCurrentSystemInfo().getClusterBackends(SystemInfoService.DEFAULT_CLUSTER); + final List defaultClusterBackends = Catalog.getCurrentSystemInfo() + .getClusterBackends(SystemInfoService.DEFAULT_CLUSTER); for (Backend backend : defaultClusterBackends) { backendList.add(backend.getId()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java index 78d200b9f76efd1..9803f83cc3486c1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java +++ b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java @@ -87,6 +87,7 @@ import org.apache.doris.persist.ReplicaPersistInfo; import org.apache.doris.persist.RoutineLoadOperation; import org.apache.doris.persist.SetReplicaStatusOperationLog; +import org.apache.doris.persist.TableAddOrDropColumnsInfo; import org.apache.doris.persist.TableInfo; import org.apache.doris.persist.TablePropertyInfo; import org.apache.doris.persist.TruncateTableInfo; @@ -662,6 +663,11 @@ public void readFields(DataInput in) throws IOException { isRead = true; break; } + case OperationType.OP_MODIFY_TABLE_ADD_OR_DROP_COLUMNS: { + data = TableAddOrDropColumnsInfo.read(in); + isRead = true; + break; + } default: { IOException e = new IOException(); LOG.error("UNKNOWN Operation Type {}", opCode, e); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java index 649e030a50123b0..0ca079347507492 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java @@ -65,6 +65,7 @@ import org.apache.doris.task.AgentTaskExecutor; import org.apache.doris.task.AgentTaskQueue; import org.apache.doris.task.PushTask; +import org.apache.doris.thrift.TColumn; import org.apache.doris.thrift.TPriority; import org.apache.doris.thrift.TPushType; import org.apache.doris.thrift.TTaskType; @@ -238,6 +239,11 @@ public void process(DeleteStmt stmt) throws DdlException, QueryStateException { long indexId = index.getId(); int schemaHash = olapTable.getSchemaHashByIndexId(indexId); + List columnsDesc = new ArrayList(); + for (Column column : olapTable.getSchemaByIndexId(indexId)) { + columnsDesc.add(column.toThrift()); + } + for (Tablet tablet : index.getTablets()) { long tabletId = tablet.getId(); @@ -260,7 +266,8 @@ public void process(DeleteStmt stmt) throws DdlException, QueryStateException { TTaskType.REALTIME_PUSH, transactionId, Catalog.getCurrentGlobalTransactionMgr() - .getTransactionIDGenerator().getNextTransactionId()); + .getTransactionIDGenerator().getNextTransactionId(), + columnsDesc); pushTask.setIsSchemaChanging(false); pushTask.setCountDownLatch(countDownLatch); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java index 0faf356029840af..954db1d971b24ea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java @@ -18,6 +18,7 @@ package org.apache.doris.load; import org.apache.doris.catalog.Catalog; +import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.MaterializedIndex; import org.apache.doris.catalog.MaterializedIndex.IndexExtState; @@ -41,6 +42,7 @@ import org.apache.doris.task.MasterTask; import org.apache.doris.task.MasterTaskExecutor; import org.apache.doris.task.PushTask; +import org.apache.doris.thrift.TColumn; import org.apache.doris.thrift.TPriority; import org.apache.doris.thrift.TPushType; import org.apache.doris.thrift.TTaskType; @@ -434,6 +436,10 @@ private Set submitPushTasks(LoadJob job, Database db) { } int schemaHash = tableLoadInfo.getIndexSchemaHash(indexId); + List columnsDesc = new ArrayList(); + for (Column column : table.getSchemaByIndexId(indexId)) { + columnsDesc.add(column.toThrift()); + } short quorumNum = (short) (replicationNum / 2 + 1); for (Tablet tablet : index.getTablets()) { long tabletId = tablet.getId(); @@ -471,7 +477,7 @@ private Set submitPushTasks(LoadJob job, Database db) { job.getId(), type, job.getConditions(), needDecompress, job.getPriority(), TTaskType.REALTIME_PUSH, job.getTransactionId(), Catalog.getCurrentGlobalTransactionMgr() - .getTransactionIDGenerator().getNextTransactionId()); + .getTransactionIDGenerator().getNextTransactionId(), columnsDesc); pushTask.setIsSchemaChanging(autoLoadToTwoTablet); if (AgentTaskQueue.addTask(pushTask)) { batchTask.addTask(pushTask); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java index b9ec64ea7f4a44d..59a03c575358644 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java @@ -73,6 +73,7 @@ import org.apache.doris.thrift.TBrokerRangeDesc; import org.apache.doris.thrift.TBrokerScanRange; import org.apache.doris.thrift.TBrokerScanRangeParams; +import org.apache.doris.thrift.TColumn; import org.apache.doris.thrift.TDescriptorTable; import org.apache.doris.thrift.TFileFormatType; import org.apache.doris.thrift.TFileType; @@ -101,6 +102,7 @@ import java.io.DataOutput; import java.io.File; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Set; @@ -109,7 +111,7 @@ * There are 4 steps in SparkLoadJob: * Step1: SparkLoadPendingTask will be created by unprotectedExecuteJob method and submit spark etl job. * Step2: LoadEtlChecker will check spark etl job status periodically - * and send push tasks to be when spark etl job is finished. + * and send push tasks to be when spark etl job is finished. * Step3: LoadLoadingChecker will check loading status periodically and commit transaction when push tasks are finished. * Step4: PublishVersionDaemon will send publish version tasks to be and finish transaction. */ @@ -149,9 +151,8 @@ public SparkLoadJob() { super(EtlJobType.SPARK); } - public SparkLoadJob(long dbId, String label, ResourceDesc resourceDesc, - OriginStatement originStmt, UserIdentity userInfo) - throws MetaNotFoundException { + public SparkLoadJob(long dbId, String label, ResourceDesc resourceDesc, OriginStatement originStmt, + UserIdentity userInfo) throws MetaNotFoundException { super(EtlJobType.SPARK, dbId, label, originStmt, userInfo); this.resourceDesc = resourceDesc; } @@ -166,6 +167,7 @@ public void setJobProperties(Map properties) throws DdlException /** * merge system conf with load stmt + * * @throws DdlException */ private void setResourceInfo() throws DdlException { @@ -210,8 +212,8 @@ protected void unprotectedExecuteJob() throws LoadException { } // create pending task - LoadTask task = new SparkLoadPendingTask(this, fileGroupAggInfo.getAggKeyToFileGroups(), - sparkResource, brokerDesc); + LoadTask task = new SparkLoadPendingTask(this, fileGroupAggInfo.getAggKeyToFileGroups(), sparkResource, + brokerDesc); task.init(); idToTasks.put(task.getSignature(), task); Catalog.getCurrentCatalog().getPendingLoadTaskScheduler().submit(task); @@ -229,19 +231,16 @@ private void onPendingTaskFinished(SparkPendingTaskAttachment attachment) { try { // check if job has been cancelled if (isTxnDone()) { - LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id) - .add("state", state) - .add("error_msg", "this task will be ignored when job is: " + state) - .build()); + LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id).add("state", state) + .add("error_msg", "this task will be ignored when job is: " + state).build()); return; } if (finishedTaskIds.contains(attachment.getTaskId())) { - LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id) - .add("task_id", attachment.getTaskId()) - .add("error_msg", "this is a duplicated callback of pending task " - + "when broker already has loading task") - .build()); + LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id).add("task_id", attachment.getTaskId()).add("error_msg", + "this is a duplicated callback of pending task " + + "when broker already has loading task") + .build()); return; } @@ -286,7 +285,7 @@ private boolean checkState(JobState expectState) { * 1. RUNNING, update etl job progress * 2. CANCELLED, cancel load job * 3. FINISHED, get the etl output file paths, update job state to LOADING and log job update info - * + *

* Send push tasks if job state changed to LOADING */ public void updateEtlStatus() throws Exception { @@ -296,8 +295,8 @@ public void updateEtlStatus() throws Exception { // get etl status SparkEtlJobHandler handler = new SparkEtlJobHandler(); - EtlStatus status = handler.getEtlJobStatus(sparkLoadAppHandle, - appId, id, etlOutputPath, sparkResource, brokerDesc); + EtlStatus status = handler.getEtlJobStatus(sparkLoadAppHandle, appId, id, etlOutputPath, sparkResource, + brokerDesc); writeLock(); try { switch (status.getState()) { @@ -364,8 +363,8 @@ private void unprotectedProcessEtlFinish(EtlStatus etlStatus, SparkEtlJobHandler unprotectedPrepareLoadingInfos(); } - private void unprotectedUpdateToLoadingState(EtlStatus etlStatus, - Map filePathToSize) throws LoadException { + private void unprotectedUpdateToLoadingState(EtlStatus etlStatus, Map filePathToSize) + throws LoadException { try { for (Map.Entry entry : filePathToSize.entrySet()) { String filePath = entry.getKey(); @@ -428,8 +427,8 @@ private Set submitPushTasks() throws UserException { AgentBatchTask batchTask = new AgentBatchTask(); boolean hasLoadPartitions = false; Set totalTablets = Sets.newHashSet(); - List tableList = - db.getTablesOnIdOrderOrThrowException(Lists.newArrayList(tableToLoadPartitions.keySet())); + List tableList = db.getTablesOnIdOrderOrThrowException( + Lists.newArrayList(tableToLoadPartitions.keySet())); MetaLockUtils.readLockTables(tableList); try { writeLock(); @@ -454,20 +453,26 @@ private Set submitPushTasks() throws UserException { } hasLoadPartitions = true; - int quorumReplicaNum = olapTable.getPartitionInfo() - .getReplicaAllocation(partitionId).getTotalReplicaNum() / 2 + 1; + int quorumReplicaNum = + olapTable.getPartitionInfo().getReplicaAllocation(partitionId).getTotalReplicaNum() / 2 + + 1; List indexes = partition.getMaterializedIndices(IndexExtState.ALL); for (MaterializedIndex index : indexes) { long indexId = index.getId(); int schemaHash = indexToSchemaHash.get(indexId); + List columnsDesc = new ArrayList(); + for (Column column : olapTable.getSchemaByIndexId(indexId)) { + columnsDesc.add(column.toThrift()); + } + int bucket = 0; for (Tablet tablet : index.getTablets()) { long tabletId = tablet.getId(); totalTablets.add(tabletId); String tabletMetaStr = String.format("%d.%d.%d.%d.%d", olapTable.getId(), partitionId, - indexId, bucket++, schemaHash); + indexId, bucket++, schemaHash); Set tabletAllReplicas = Sets.newHashSet(); Set tabletFinishedReplicas = Sets.newHashSet(); for (Replica replica : tablet.getReplicas()) { @@ -482,8 +487,8 @@ private Set submitPushTasks() throws UserException { PushBrokerReaderParams params = getPushBrokerReaderParams(olapTable, indexId); // deep copy TBrokerScanRange because filePath and fileSize will be updated // in different tablet push task - TBrokerScanRange tBrokerScanRange - = new TBrokerScanRange(params.tBrokerScanRange); + TBrokerScanRange tBrokerScanRange = new TBrokerScanRange( + params.tBrokerScanRange); // update filePath fileSize TBrokerRangeDesc tBrokerRangeDesc = tBrokerScanRange.getRanges().get(0); tBrokerRangeDesc.setPath(""); @@ -497,22 +502,21 @@ private Set submitPushTasks() throws UserException { // update broker address Backend backend = Catalog.getCurrentCatalog().getCurrentSystemInfo() .getBackend(backendId); - FsBroker fsBroker = Catalog.getCurrentCatalog().getBrokerMgr().getBroker( - brokerDesc.getName(), backend.getHost()); - tBrokerScanRange.getBrokerAddresses().add( - new TNetworkAddress(fsBroker.ip, fsBroker.port)); + FsBroker fsBroker = Catalog.getCurrentCatalog().getBrokerMgr() + .getBroker(brokerDesc.getName(), backend.getHost()); + tBrokerScanRange.getBrokerAddresses() + .add(new TNetworkAddress(fsBroker.ip, fsBroker.port)); LOG.debug("push task for replica {}, broker {}:{}," + " backendId {}, filePath {}, fileSize {}", - replicaId, fsBroker.ip, fsBroker.port, backendId, - tBrokerRangeDesc.path, + replicaId, fsBroker.ip, + fsBroker.port, backendId, tBrokerRangeDesc.path, tBrokerRangeDesc.file_size); - PushTask pushTask = new PushTask( - backendId, dbId, olapTable.getId(), partitionId, indexId, tabletId, - replicaId, schemaHash, 0, id, TPushType.LOAD_V2, - TPriority.NORMAL, transactionId, taskSignature, - tBrokerScanRange, params.tDescriptorTable); + PushTask pushTask = new PushTask(backendId, dbId, olapTable.getId(), + partitionId, indexId, tabletId, replicaId, schemaHash, 0, id, + TPushType.LOAD_V2, TPriority.NORMAL, transactionId, taskSignature, + tBrokerScanRange, params.tDescriptorTable, columnsDesc); if (AgentTaskQueue.addTask(pushTask)) { batchTask.addTask(pushTask); if (!tabletToSentReplicaPushTask.containsKey(tabletId)) { @@ -548,11 +552,8 @@ private Set submitPushTasks() throws UserException { } if (!hasLoadPartitions) { - String errMsg = new LogBuilder(LogKey.LOAD_JOB, id) - .add("database_id", dbId) - .add("label", label) - .add("error_msg", "all partitions have no load data") - .build(); + String errMsg = new LogBuilder(LogKey.LOAD_JOB, id).add("database_id", dbId).add("label", label) + .add("error_msg", "all partitions have no load data").build(); throw new LoadException(errMsg); } @@ -632,19 +633,16 @@ public void updateLoadingStatus() throws UserException { } private void tryCommitJob() throws UserException { - LOG.info(new LogBuilder(LogKey.LOAD_JOB, id) - .add("txn_id", transactionId) - .add("msg", "Load job try to commit txn") - .build()); + LOG.info(new LogBuilder(LogKey.LOAD_JOB, id).add("txn_id", transactionId) + .add("msg", "Load job try to commit txn").build()); Database db = getDb(); List

tableList = db.getTablesOnIdOrderOrThrowException( Lists.newArrayList(tableToLoadPartitions.keySet())); MetaLockUtils.writeLockTablesOrMetaException(tableList); try { - Catalog.getCurrentGlobalTransactionMgr().commitTransaction( - dbId, tableList, transactionId, commitInfos, - new LoadJobFinalOperation(id, loadingStatus, progress, loadStartTimestamp, - finishTimestamp, state, failMsg)); + Catalog.getCurrentGlobalTransactionMgr().commitTransaction(dbId, tableList, transactionId, commitInfos, + new LoadJobFinalOperation(id, loadingStatus, progress, loadStartTimestamp, finishTimestamp, state, + failMsg)); } catch (TabletQuorumFailedException e) { // retry in next loop } finally { @@ -785,9 +783,8 @@ public void readFields(DataInput in) throws IOException { * log load job update info when job state changed to etl or loading */ private void unprotectedLogUpdateStateInfo() { - SparkLoadJobStateUpdateInfo info = new SparkLoadJobStateUpdateInfo( - id, state, transactionId, sparkLoadAppHandle, etlStartTimestamp, appId, etlOutputPath, - loadStartTimestamp, tabletMetaToFileInfo); + SparkLoadJobStateUpdateInfo info = new SparkLoadJobStateUpdateInfo(id, state, transactionId, sparkLoadAppHandle, + etlStartTimestamp, appId, etlOutputPath, loadStartTimestamp, tabletMetaToFileInfo); Catalog.getCurrentCatalog().getEditLog().logUpdateLoadJob(info); } @@ -809,8 +806,8 @@ public void replayUpdateStateInfo(LoadJobStateUpdateInfo info) { unprotectedPrepareLoadingInfos(); break; default: - LOG.warn("replay update load job state info failed. error: wrong state. job id: {}, state: {}", - id, state); + LOG.warn("replay update load job state info failed. error: wrong state. job id: {}, state: {}", id, + state); break; } } @@ -830,10 +827,9 @@ public static class SparkLoadJobStateUpdateInfo extends LoadJobStateUpdateInfo { @SerializedName(value = "tabletMetaToFileInfo") private Map> tabletMetaToFileInfo; - public SparkLoadJobStateUpdateInfo( - long jobId, JobState state, long transactionId, SparkLoadAppHandle sparkLoadAppHandle, - long etlStartTimestamp, String appId, String etlOutputPath, long loadStartTimestamp, - Map> tabletMetaToFileInfo) { + public SparkLoadJobStateUpdateInfo(long jobId, JobState state, long transactionId, + SparkLoadAppHandle sparkLoadAppHandle, long etlStartTimestamp, String appId, String etlOutputPath, + long loadStartTimestamp, Map> tabletMetaToFileInfo) { super(jobId, state, transactionId, loadStartTimestamp); this.sparkLoadAppHandle = sparkLoadAppHandle; this.etlStartTimestamp = etlStartTimestamp; @@ -867,7 +863,7 @@ public Map> getTabletMetaToFileInfo() { * Params for be push broker reader * 1. TBrokerScanRange: file path and size, broker address, tranform expr * 2. TDescriptorTable: src and dest SlotDescriptors, src and dest tupleDescriptors - * + *

* These params are sent to Be through push task */ private static class PushBrokerReaderParams { @@ -896,7 +892,7 @@ public void init(List columns, BrokerDesc brokerDesc) throws UserExcepti } private void initTBrokerScanRange(DescriptorTable descTable, TupleDescriptor destTupleDesc, - List columns, BrokerDesc brokerDesc) throws AnalysisException { + List columns, BrokerDesc brokerDesc) throws AnalysisException { // scan range params TBrokerScanRangeParams params = new TBrokerScanRangeParams(); params.setStrictMode(false); diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index 3e0b53197003650..03bc694ae33fadc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -180,47 +180,45 @@ public static void loadJournal(Catalog catalog, JournalEntity journal) { } case OperationType.OP_CREATE_TABLE: { CreateTableInfo info = (CreateTableInfo) journal.getData(); - LOG.info("Begin to unprotect create table. db = " - + info.getDbName() + " table = " + info.getTable().getId()); + LOG.info("Begin to unprotect create table. db = " + info.getDbName() + " table = " + info.getTable() + .getId()); catalog.replayCreateTable(info.getDbName(), info.getTable()); break; } case OperationType.OP_ALTER_EXTERNAL_TABLE_SCHEMA: { RefreshExternalTableInfo info = (RefreshExternalTableInfo) journal.getData(); - LOG.info("Begin to unprotect alter external table schema. db = " - + info.getDbName() + " table = " + info.getTableName()); + LOG.info("Begin to unprotect alter external table schema. db = " + info.getDbName() + " table = " + + info.getTableName()); catalog.replayAlterExternalTableSchema(info.getDbName(), info.getTableName(), info.getNewSchema()); break; } case OperationType.OP_DROP_TABLE: { DropInfo info = (DropInfo) journal.getData(); - Database db = - Catalog.getCurrentInternalCatalog().getDbOrMetaException(info.getDbId()); - LOG.info("Begin to unprotect drop table. db = " - + db.getFullName() + " table = " + info.getTableId()); + Database db = Catalog.getCurrentInternalCatalog().getDbOrMetaException(info.getDbId()); + LOG.info("Begin to unprotect drop table. db = " + db.getFullName() + " table = " + + info.getTableId()); catalog.replayDropTable(db, info.getTableId(), info.isForceDrop()); break; } case OperationType.OP_ADD_PARTITION: { PartitionPersistInfo info = (PartitionPersistInfo) journal.getData(); - LOG.info("Begin to unprotect add partition. db = " + info.getDbId() - + " table = " + info.getTableId() - + " partitionName = " + info.getPartition().getName()); + LOG.info( + "Begin to unprotect add partition. db = " + info.getDbId() + " table = " + info.getTableId() + + " partitionName = " + info.getPartition().getName()); catalog.replayAddPartition(info); break; } case OperationType.OP_DROP_PARTITION: { DropPartitionInfo info = (DropPartitionInfo) journal.getData(); - LOG.info("Begin to unprotect drop partition. db = " + info.getDbId() - + " table = " + info.getTableId() - + " partitionName = " + info.getPartitionName()); + LOG.info("Begin to unprotect drop partition. db = " + info.getDbId() + " table = " + + info.getTableId() + " partitionName = " + info.getPartitionName()); catalog.replayDropPartition(info); break; } case OperationType.OP_MODIFY_PARTITION: { ModifyPartitionInfo info = (ModifyPartitionInfo) journal.getData(); - LOG.info("Begin to unprotect modify partition. db = " + info.getDbId() - + " table = " + info.getTableId() + " partitionId = " + info.getPartitionId()); + LOG.info("Begin to unprotect modify partition. db = " + info.getDbId() + " table = " + + info.getTableId() + " partitionId = " + info.getPartitionId()); catalog.getAlterInstance().replayModifyPartition(info); break; } @@ -285,8 +283,9 @@ public static void loadJournal(Catalog catalog, JournalEntity journal) { case OperationType.OP_BATCH_DROP_ROLLUP: { BatchDropInfo batchDropInfo = (BatchDropInfo) journal.getData(); for (long indexId : batchDropInfo.getIndexIdSet()) { - catalog.getMaterializedViewHandler().replayDropRollup(new DropInfo(batchDropInfo.getDbId(), - batchDropInfo.getTableId(), indexId, false), catalog); + catalog.getMaterializedViewHandler().replayDropRollup( + new DropInfo(batchDropInfo.getDbId(), batchDropInfo.getTableId(), indexId, false), + catalog); } break; } @@ -846,6 +845,11 @@ public static void loadJournal(Catalog catalog, JournalEntity journal) { catalog.getDataSourceMgr().replayAlterCatalogProps(log); break; } + case OperationType.OP_MODIFY_TABLE_ADD_OR_DROP_COLUMNS: { + final TableAddOrDropColumnsInfo info = (TableAddOrDropColumnsInfo) journal.getData(); + catalog.getSchemaChangeHandler().replayModifyTableAddOrDropColumns(info); + break; + } default: { IOException e = new IOException(); LOG.error("UNKNOWN Operation Type {}", opCode, e); @@ -935,13 +939,13 @@ private synchronized void logEdit(short op, Writable writable) { } if (LOG.isDebugEnabled()) { - LOG.debug("nextId = {}, numTransactions = {}, totalTimeTransactions = {}, op = {}", - txId, numTransactions, totalTimeTransactions, op); + LOG.debug("nextId = {}, numTransactions = {}, totalTimeTransactions = {}, op = {}", txId, numTransactions, + totalTimeTransactions, op); } if (txId >= Config.edit_log_roll_num) { - LOG.info("txId {} is equal to or larger than edit_log_roll_num {}, will roll edit.", - txId, Config.edit_log_roll_num); + LOG.info("txId {} is equal to or larger than edit_log_roll_num {}, will roll edit.", txId, + Config.edit_log_roll_num); rollEditLog(); txId = 0; } @@ -1472,4 +1476,8 @@ public void logDatasourceLog(short id, CatalogLog log) { public Journal getJournal() { return this.journal; } + + public void logModifyTableAddOrDropColumns(TableAddOrDropColumnsInfo info) { + logEdit(OperationType.OP_MODIFY_TABLE_ADD_OR_DROP_COLUMNS, info); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java index 79d3cf63a09f392..047714d0f070dc0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java @@ -69,6 +69,9 @@ public class OperationType { public static final short OP_MODIFY_COMMENT = 126; public static final short OP_MODIFY_TABLE_ENGINE = 127; + //schema change for add and drop columns + public static final short OP_MODIFY_TABLE_ADD_OR_DROP_COLUMNS = 128; + // 30~39 130~139 230~239 ... // load job for only hadoop load public static final short OP_LOAD_START = 30; diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/TableAddOrDropColumnsInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/TableAddOrDropColumnsInfo.java new file mode 100644 index 000000000000000..6e644f6a3b50abc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/TableAddOrDropColumnsInfo.java @@ -0,0 +1,115 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.persist; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Index; +import org.apache.doris.common.io.Text; +import org.apache.doris.common.io.Writable; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.gson.annotations.SerializedName; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * PersistInfo for Table properties + */ +public class TableAddOrDropColumnsInfo implements Writable { + @SerializedName(value = "dbId") + private long dbId; + @SerializedName(value = "tableId") + private long tableId; + @SerializedName(value = "indexSchemaMap") + private Map> indexSchemaMap; + @SerializedName(value = "indexes") + private List indexes; + @SerializedName(value = "jobId") + private long jobId; + + public TableAddOrDropColumnsInfo(long dbId, long tableId, + Map> indexSchemaMap, List indexes, long jobId) { + this.dbId = dbId; + this.tableId = tableId; + this.indexSchemaMap = indexSchemaMap; + this.indexes = indexes; + this.jobId = jobId; + } + + public long getDbId() { + return dbId; + } + + public long getTableId() { + return tableId; + } + + public Map> getIndexSchemaMap() { + return indexSchemaMap; + } + + public List getIndexes() { + return indexes; + } + + public long getJobId() { + return jobId; + } + + @Override + public void write(DataOutput out) throws IOException { + Text.writeString(out, GsonUtils.GSON.toJson(this)); + } + + public static TableAddOrDropColumnsInfo read(DataInput in) throws IOException { + return GsonUtils.GSON.fromJson(Text.readString(in), TableAddOrDropColumnsInfo.class); + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + + if (!(obj instanceof TableAddOrDropColumnsInfo)) { + return false; + } + + TableAddOrDropColumnsInfo info = (TableAddOrDropColumnsInfo) obj; + + return (dbId == info.dbId && tableId == tableId + && indexSchemaMap.equals(info.indexSchemaMap) && indexes.equals(info.indexes) + && jobId == info.jobId); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(" dbId: ").append(dbId); + sb.append(" tableId: ").append(tableId); + sb.append(" indexSchemaMap: ").append(indexSchemaMap); + sb.append(" indexes: ").append(indexes); + sb.append(" jobId: ").append(jobId); + return sb.toString(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java index c5fff0567c52628..459414640d94049 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java @@ -58,6 +58,7 @@ import org.apache.doris.statistics.StatisticalType; import org.apache.doris.statistics.StatsRecursiveDerive; import org.apache.doris.system.Backend; +import org.apache.doris.thrift.TColumn; import org.apache.doris.thrift.TExplainLevel; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TOlapScanNode; @@ -97,8 +98,10 @@ public class OlapScanNode extends ScanNode { private List result = new ArrayList<>(); /* - * When the field value is ON, the storage engine can return the data directly without pre-aggregation. - * When the field value is OFF, the storage engine needs to aggregate the data before returning to scan node. + * When the field value is ON, the storage engine can return the data directly + * without pre-aggregation. + * When the field value is OFF, the storage engine needs to aggregate the data + * before returning to scan node. * For example: * Aggregate table: k1, k2, v1 sum * Field value is ON @@ -111,20 +114,26 @@ public class OlapScanNode extends ScanNode { * This aggregation info is null. * Query2: select k1, min(v1) from table group by k1 * This aggregation function in query is min which different from the schema. - * So the data stored in storage engine need to be merged firstly before returning to scan node. + * So the data stored in storage engine need to be merged firstly before + * returning to scan node. * * There are currently two places to modify this variable: * 1. The turnOffPreAgg() method of SingleNodePlanner. - * This method will only be called on the left deepest OlapScanNode the plan tree, - * while other nodes are false by default (because the Aggregation operation is executed after Join, - * we cannot judge whether other OlapScanNodes can close the pre-aggregation). - * So even the Duplicate key table, if it is not the left deepest node, it will remain false too. + * This method will only be called on the left deepest OlapScanNode the plan + * tree, + * while other nodes are false by default (because the Aggregation operation is + * executed after Join, + * we cannot judge whether other OlapScanNodes can close the pre-aggregation). + * So even the Duplicate key table, if it is not the left deepest node, it will + * remain false too. * - * 2. After MaterializedViewSelector selects the materialized view, the updateScanRangeInfoByNewMVSelector()\ - * method of OlapScanNode may be called to update this variable. - * This call will be executed on all ScanNodes in the plan tree. In this step, - * for the DuplicateKey table, the variable will be set to true. - * See comment of "isPreAggregation" variable in MaterializedViewSelector for details. + * 2. After MaterializedViewSelector selects the materialized view, the + * updateScanRangeInfoByNewMVSelector()\ + * method of OlapScanNode may be called to update this variable. + * This call will be executed on all ScanNodes in the plan tree. In this step, + * for the DuplicateKey table, the variable will be set to true. + * See comment of "isPreAggregation" variable in MaterializedViewSelector for + * details. */ private boolean isPreAggregation = false; private String reasonOfPreAggregation = null; @@ -144,7 +153,8 @@ public class OlapScanNode extends ScanNode { private HashSet scanBackendIds = new HashSet<>(); private Map tabletId2BucketSeq = Maps.newHashMap(); - // a bucket seq may map to many tablets, and each tablet has a TScanRangeLocations. + // a bucket seq may map to many tablets, and each tablet has a + // TScanRangeLocations. public ArrayListMultimap bucketSeq2locations = ArrayListMultimap.create(); // Constructs node to scan given data files of table 'tbl'. @@ -205,36 +215,45 @@ public void setSelectedPartitionIds(Collection selectedPartitionIds) { } /** - * The function is used to directly select the index id of the base table as the selectedIndexId. - * It makes sure that the olap scan node must scan the base data rather than scan the materialized view data. + * The function is used to directly select the index id of the base table as the + * selectedIndexId. + * It makes sure that the olap scan node must scan the base data rather than + * scan the materialized view data. * * This function is mainly used to update stmt. * Update stmt also needs to scan data like normal queries. * But its syntax is different from ordinary queries, - * so planner cannot use the logic of query to automatically match the best index id. - * So, here it need to manually specify the index id to scan the base table directly. + * so planner cannot use the logic of query to automatically match the best + * index id. + * So, here it need to manually specify the index id to scan the base table + * directly. */ public void useBaseIndexId() { this.selectedIndexId = olapTable.getBaseIndexId(); } /** - * This method is mainly used to update scan range info in OlapScanNode by the new materialized selector. + * This method is mainly used to update scan range info in OlapScanNode by the + * new materialized selector. * Situation1: - * If the new scan range is same as the old scan range which determined by the old materialized selector, + * If the new scan range is same as the old scan range which determined by the + * old materialized selector, * the scan range will not be changed. *

* Situation2: Scan range is difference. The type of table is duplicated. * The new scan range is used directly. - * The reason is that the old selector does not support SPJ<->SPJG, so the result of old one must be incorrect. + * The reason is that the old selector does not support SPJ<->SPJG, so the + * result of old one must be incorrect. *

* Situation3: Scan range is difference. The type of table is aggregated. * The new scan range is different from the old one. * If the test_materialized_view is set to true, an error will be reported. * The query will be cancelled. *

- * Situation4: Scan range is difference. The type of table is aggregated. `test_materialized_view` is set to false. - * The result of the old version selector will be selected. Print the warning log + * Situation4: Scan range is difference. The type of table is aggregated. + * `test_materialized_view` is set to false. + * The result of the old version selector will be selected. Print the warning + * log * * @param selectedIndexId * @param isPreAggregation @@ -257,8 +276,7 @@ public void updateScanRangeInfoByNewMVSelector(long selectedIndexId, String scanRangeInfo = stringBuilder.toString(); String situation; boolean update; - CHECK: - { // CHECKSTYLE IGNORE THIS LINE + CHECK: { // CHECKSTYLE IGNORE THIS LINE if (olapTable.getKeysType() == KeysType.DUP_KEYS) { situation = "The key type of table is duplicate."; update = true; @@ -298,12 +316,14 @@ public void updateScanRangeInfoByNewMVSelector(long selectedIndexId, /** * In some situation, the column type between base and mv is different. - * If mv selector selects the mv index, the type of column should be changed to the type of mv column. + * If mv selector selects the mv index, the type of column should be changed to + * the type of mv column. * For example: * base table: k1 int, k2 int * mv table: k1 int, k2 bigint sum * The type of `k2` column between base and mv is different. - * When mv selector selects the mv table to scan, the type of column should be changed to bigint in here. + * When mv selector selects the mv table to scan, the type of column should be + * changed to bigint in here. * Currently, only `SUM` aggregate type could match this changed. */ private void updateColumnType() { @@ -348,9 +368,12 @@ public void init(Analyzer analyzer) throws UserException { /** * Compute InAccurate cardinality before mv selector and tablet pruning. - * - Accurate statistical information relies on the selector of materialized views and bucket reduction. - * - However, Those both processes occur after the reorder algorithm is completed. - * - When Join reorder is turned on, the cardinality must be calculated before the reorder algorithm. + * - Accurate statistical information relies on the selector of materialized + * views and bucket reduction. + * - However, Those both processes occur after the reorder algorithm is + * completed. + * - When Join reorder is turned on, the cardinality must be calculated before + * the reorder algorithm. * - So only an inaccurate cardinality can be calculated here. */ mockRowCountInStatistic(); @@ -390,7 +413,8 @@ public void mockRowCountInStatistic() { public void finalize(Analyzer analyzer) throws UserException { LOG.debug("OlapScanNode get scan range locations. Tuple: {}", desc); /** - * If JoinReorder is turned on, it will be calculated init(), and this value is not accurate. + * If JoinReorder is turned on, it will be calculated init(), and this value is + * not accurate. * In the following logic, cardinality will be accurately calculated again. * So here we need to reset the value of cardinality. */ @@ -402,7 +426,8 @@ public void finalize(Analyzer analyzer) throws UserException { } catch (AnalysisException e) { throw new UserException(e.getMessage()); } - // Relatively accurate cardinality according to ScanRange in getScanRangeLocations + // Relatively accurate cardinality according to ScanRange in + // getScanRangeLocations computeStats(analyzer); computeNumNodes(); } @@ -420,7 +445,8 @@ public void computeStats(Analyzer analyzer) throws UserException { avgRowSize = totalBytes / (float) cardinality * COMPRESSION_RATIO; capCardinalityAtLimit(); } - // when node scan has no data, cardinality should be 0 instead of a invalid value after computeStats() + // when node scan has no data, cardinality should be 0 instead of a invalid + // value after computeStats() cardinality = cardinality == -1 ? 0 : cardinality; // update statsDeriveResult for real statistics @@ -435,7 +461,8 @@ protected void computeNumNodes() { if (cardinality > 0) { numNodes = scanBackendIds.size(); } - // even current node scan has no data,at least on backend will be assigned when the fragment actually execute + // even current node scan has no data,at least on backend will be assigned when + // the fragment actually execute numNodes = numNodes <= 0 ? 1 : numNodes; } @@ -472,10 +499,10 @@ private Collection partitionPrune(PartitionInfo partitionInfo, } else if (partitionInfo.getType() == PartitionType.LIST) { if (analyzer.partitionPruneV2Enabled()) { partitionPruner = new ListPartitionPrunerV2(keyItemMap, partitionInfo.getPartitionColumns(), - columnNameToRange); + columnNameToRange); } else { partitionPruner = new ListPartitionPruner(keyItemMap, - partitionInfo.getPartitionColumns(), columnFilters); + partitionInfo.getPartitionColumns(), columnFilters); } } return partitionPruner.prune(); @@ -504,7 +531,7 @@ private Collection distributionPrune( } private void addScanRangeLocations(Partition partition, - List tablets) throws UserException { + List tablets) throws UserException { long visibleVersion = partition.getVisibleVersion(); String visibleVersionStr = String.valueOf(visibleVersion); @@ -615,8 +642,8 @@ private void computePartitionInfo() throws AnalysisException { } } else { selectedPartitionIds = selectedPartitionIds.stream() - .filter(id -> olapTable.getPartition(id).hasData()) - .collect(Collectors.toList()); + .filter(id -> olapTable.getPartition(id).hasData()) + .collect(Collectors.toList()); } selectedPartitionNum = selectedPartitionIds.size(); @@ -635,9 +662,10 @@ public void selectBestRollupByRollupSelector(Analyzer analyzer) throws UserExcep // Step2: select best rollup long start = System.currentTimeMillis(); if (olapTable.getKeysType() == KeysType.DUP_KEYS) { - //This function is compatible with the INDEX selection logic of ROLLUP, - //so the Duplicate table here returns base index directly - //and the selection logic of materialized view is selected in "MaterializedViewSelector" + // This function is compatible with the INDEX selection logic of ROLLUP, + // so the Duplicate table here returns base index directly + // and the selection logic of materialized view is selected in + // "MaterializedViewSelector" selectedIndexId = olapTable.getBaseIndexId(); LOG.debug("The best index will be selected later in mv selector"); return; @@ -759,18 +787,23 @@ public int getNumInstances() { protected void toThrift(TPlanNode msg) { List keyColumnNames = new ArrayList(); List keyColumnTypes = new ArrayList(); + List columnsDesc = new ArrayList(); + if (selectedIndexId != -1) { - for (Column col : olapTable.getSchemaByIndexId(selectedIndexId)) { - if (!col.isKey()) { - break; + for (Column col : olapTable.getSchemaByIndexId(selectedIndexId, true)) { + TColumn tColumn = col.toThrift(); + col.setIndexFlag(tColumn, olapTable.getIndexes()); + columnsDesc.add(tColumn); + if ((Util.showHiddenColumns() || (!Util.showHiddenColumns() && col.isVisible())) && col.isKey()) { + keyColumnNames.add(col.getName()); + keyColumnTypes.add(col.getDataType().toThrift()); } - keyColumnNames.add(col.getName()); - keyColumnTypes.add(col.getDataType().toThrift()); } } + msg.node_type = TPlanNodeType.OLAP_SCAN_NODE; - msg.olap_scan_node = - new TOlapScanNode(desc.getId().asInt(), keyColumnNames, keyColumnTypes, isPreAggregation); + msg.olap_scan_node = new TOlapScanNode(desc.getId().asInt(), keyColumnNames, keyColumnTypes, isPreAggregation, + columnsDesc); if (null != sortColumn) { msg.olap_scan_node.setSortColumn(sortColumn); } @@ -835,7 +868,6 @@ public TupleId getTupleId() { return desc.getId(); } - private boolean isEquivalenceExpr(Expr expr) { if (expr instanceof InPredicate) { return true; @@ -916,10 +948,10 @@ private void filterDeletedRows(Analyzer analyzer) throws AnalysisException { } /* - Although sometimes the scan range only involves one instance, - the data distribution cannot be set to UNPARTITIONED here. - The reason is that @coordinator will not set the scan range for the fragment, - when data partition of fragment is UNPARTITIONED. + * Although sometimes the scan range only involves one instance, + * the data distribution cannot be set to UNPARTITIONED here. + * The reason is that @coordinator will not set the scan range for the fragment, + * when data partition of fragment is UNPARTITIONED. */ public DataPartition constructInputPartitionByDistributionInfo() throws UserException { ColocateTableIndex colocateTableIndex = Catalog.getCurrentColocateIndex(); 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 3b462f6d20ea3c7..1c69425ec886db6 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 @@ -45,6 +45,7 @@ import org.apache.doris.common.UserException; import org.apache.doris.system.Backend; import org.apache.doris.system.SystemInfoService; +import org.apache.doris.thrift.TColumn; import org.apache.doris.thrift.TDataSink; import org.apache.doris.thrift.TDataSinkType; import org.apache.doris.thrift.TExplainLevel; @@ -93,8 +94,8 @@ public OlapTableSink(OlapTable dstTable, TupleDescriptor tupleDescriptor, List pair : table.getIndexIdToMeta().entrySet()) { MaterializedIndexMeta indexMeta = pair.getValue(); List columns = Lists.newArrayList(); + List columnsDesc = Lists.newArrayList(); columns.addAll(indexMeta.getSchema().stream().map(Column::getName).collect(Collectors.toList())); + for (Column column : indexMeta.getSchema()) { + TColumn tColumn = column.toThrift(); + column.setIndexFlag(tColumn, table.getIndexes()); + columnsDesc.add(tColumn); + } TOlapTableIndexSchema indexSchema = new TOlapTableIndexSchema(pair.getKey(), columns, - indexMeta.getSchemaHash()); + indexMeta.getSchemaHash(), columnsDesc); schemaParam.addToIndexes(indexSchema); } return schemaParam; @@ -276,8 +283,7 @@ private TOlapTablePartitionParam createPartition(long dbId, OlapTable table) thr tPartition.setNumBuckets(index.getTablets().size()); } partitionParam.addToPartitions(tPartition); - partitionParam.setDistributedColumns( - getDistColumns(partition.getDistributionInfo())); + partitionParam.setDistributedColumns(getDistColumns(partition.getDistributionInfo())); break; } default: { @@ -328,13 +334,12 @@ private TOlapTableLocationParam createLocation(OlapTable table) throws UserExcep for (Tablet tablet : index.getTablets()) { Multimap bePathsMap = tablet.getNormalReplicaBackendPathMap(); if (bePathsMap.keySet().size() < quorum) { - throw new UserException(InternalErrorCode.REPLICA_FEW_ERR, "tablet " + tablet.getId() - + " has few replicas: " + bePathsMap.keySet().size() - + ", alive backends: [" + StringUtils.join(bePathsMap.keySet(), ",") - + "]"); + throw new UserException(InternalErrorCode.REPLICA_FEW_ERR, + "tablet " + tablet.getId() + " has few replicas: " + bePathsMap.keySet().size() + + ", alive backends: [" + StringUtils.join(bePathsMap.keySet(), ",") + "]"); } - locationParam.addToTablets(new TTabletLocation(tablet.getId(), - Lists.newArrayList(bePathsMap.keySet()))); + locationParam.addToTablets( + new TTabletLocation(tablet.getId(), Lists.newArrayList(bePathsMap.keySet()))); allBePathsMap.putAll(bePathsMap); } } 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 8f62a135ec3209f..e5eed8c2507b280 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 @@ -211,8 +211,8 @@ public TGetTablesResult getTableNames(TGetTablesParams params) throws TException if (db != null) { for (String tableName : db.getTableNamesWithLock()) { LOG.debug("get table: {}, wait to check", tableName); - if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(currentUser, params.db, - tableName, PrivPredicate.SHOW)) { + if (!Catalog.getCurrentCatalog().getAuth() + .checkTblPriv(currentUser, params.db, tableName, PrivPredicate.SHOW)) { continue; } @@ -270,14 +270,14 @@ public TListTableStatusResult listTableStatus(TGetTablesParams params) throws TE } } for (TableIf table : tables) { - if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(currentUser, params.db, - table.getName(), PrivPredicate.SHOW)) { + if (!Catalog.getCurrentCatalog().getAuth() + .checkTblPriv(currentUser, params.db, table.getName(), PrivPredicate.SHOW)) { continue; } table.readLock(); try { - if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(currentUser, params.db, - table.getName(), PrivPredicate.SHOW)) { + if (!Catalog.getCurrentCatalog().getAuth() + .checkTblPriv(currentUser, params.db, table.getName(), PrivPredicate.SHOW)) { continue; } @@ -386,8 +386,8 @@ public TDescribeTableResult describeTable(TDescribeTableParams params) throws TE } else { currentUser = UserIdentity.createAnalyzedUserIdentWithIp(params.user, params.user_ip); } - if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(currentUser, params.db, - params.getTableName(), PrivPredicate.SHOW)) { + if (!Catalog.getCurrentCatalog().getAuth() + .checkTblPriv(currentUser, params.db, params.getTableName(), PrivPredicate.SHOW)) { return result; } @@ -504,7 +504,7 @@ private void checkAuthCodeUuid(String dbName, long txnId, String authCodeUuid) t } private void checkPasswordAndPrivs(String cluster, String user, String passwd, String db, String tbl, - String clientIp, PrivPredicate predicate) throws AuthenticationException { + String clientIp, PrivPredicate predicate) throws AuthenticationException { final String fullUserName = ClusterNamespace.getFullName(cluster, user); final String fullDbName = ClusterNamespace.getFullName(cluster, db); @@ -533,8 +533,8 @@ public TLoadTxnBeginResult loadTxnBegin(TLoadTxnBeginRequest request) throws TEx result.setTxnId(tmpRes.getTxnId()).setDbId(tmpRes.getDbId()); } catch (DuplicatedRequestException e) { // this is a duplicate request, just return previous txn id - LOG.warn("duplicate request for stream load. request id: {}, txn: {}", - e.getDuplicatedRequestId(), e.getTxnId()); + LOG.warn("duplicate request for stream load. request id: {}, txn: {}", e.getDuplicatedRequestId(), + e.getTxnId()); result.setTxnId(e.getTxnId()); } catch (LabelAlreadyUsedException e) { status.setStatusCode(TStatusCode.LABEL_ALREADY_EXISTS); @@ -560,8 +560,8 @@ private TLoadTxnBeginResult loadTxnBeginImpl(TLoadTxnBeginRequest request, Strin } if (Strings.isNullOrEmpty(request.getAuthCodeUuid())) { - checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), - request.getTbl(), request.getUserIp(), PrivPredicate.LOAD); + checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), request.getTbl(), + request.getUserIp(), PrivPredicate.LOAD); } // check label @@ -584,10 +584,10 @@ private TLoadTxnBeginResult loadTxnBeginImpl(TLoadTxnBeginRequest request, Strin // begin long timeoutSecond = request.isSetTimeout() ? request.getTimeout() : Config.stream_load_default_timeout_second; MetricRepo.COUNTER_LOAD_ADD.increase(1L); - long txnId = Catalog.getCurrentGlobalTransactionMgr().beginTransaction( - db.getId(), Lists.newArrayList(table.getId()), request.getLabel(), request.getRequestId(), - new TxnCoordinator(TxnSourceType.BE, clientIp), - TransactionState.LoadJobSourceType.BACKEND_STREAMING, -1, timeoutSecond); + long txnId = Catalog.getCurrentGlobalTransactionMgr() + .beginTransaction(db.getId(), Lists.newArrayList(table.getId()), request.getLabel(), + request.getRequestId(), new TxnCoordinator(TxnSourceType.BE, clientIp), + TransactionState.LoadJobSourceType.BACKEND_STREAMING, -1, timeoutSecond); if (!Strings.isNullOrEmpty(request.getAuthCodeUuid())) { Catalog.getCurrentGlobalTransactionMgr().getTransactionState(db.getId(), txnId) .setAuthCode(request.getAuthCodeUuid()); @@ -631,8 +631,8 @@ private void loadTxnPreCommitImpl(TLoadTxnCommitRequest request) throws UserExce } else if (request.isSetAuthCodeUuid()) { checkAuthCodeUuid(request.getDb(), request.getTxnId(), request.getAuthCodeUuid()); } else { - checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), - request.getTbl(), request.getUserIp(), PrivPredicate.LOAD); + checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), request.getTbl(), + request.getUserIp(), PrivPredicate.LOAD); } // get database @@ -713,8 +713,8 @@ private void loadTxn2PCImpl(TLoadTxn2PCRequest request) throws UserException { List

tableList = database.getTablesOnIdOrderOrThrowException(tableIdList); for (Table table : tableList) { // check auth - checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), - table.getName(), request.getUserIp(), PrivPredicate.LOAD); + checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), table.getName(), + request.getUserIp(), PrivPredicate.LOAD); } String txnOperation = request.getOperation().trim(); @@ -767,8 +767,8 @@ private boolean loadTxnCommitImpl(TLoadTxnCommitRequest request) throws UserExce } else if (request.isSetAuthCodeUuid()) { checkAuthCodeUuid(request.getDb(), request.getTxnId(), request.getAuthCodeUuid()); } else { - checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), - request.getTbl(), request.getUserIp(), PrivPredicate.LOAD); + checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), request.getTbl(), + request.getUserIp(), PrivPredicate.LOAD); } // get database @@ -835,8 +835,8 @@ private void loadTxnRollbackImpl(TLoadTxnRollbackRequest request) throws UserExc } else if (request.isSetAuthCodeUuid()) { checkAuthCodeUuid(request.getDb(), request.getTxnId(), request.getAuthCodeUuid()); } else { - checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), - request.getTbl(), request.getUserIp(), PrivPredicate.LOAD); + checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), request.getTbl(), + request.getUserIp(), PrivPredicate.LOAD); } String dbName = ClusterNamespace.getFullName(cluster, request.getDb()); Database db; @@ -896,8 +896,8 @@ private TExecPlanFragmentParams streamLoadPutImpl(TStreamLoadPutRequest request) long timeoutMs = request.isSetThriftRpcTimeoutMs() ? request.getThriftRpcTimeoutMs() : 5000; Table table = db.getTableOrMetaException(request.getTbl(), TableType.OLAP); if (!table.tryReadLock(timeoutMs, TimeUnit.MILLISECONDS)) { - throw new UserException("get table read lock timeout, database=" - + fullDbName + ",table=" + table.getName()); + throw new UserException( + "get table read lock timeout, database=" + fullDbName + ",table=" + table.getName()); } try { StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request); @@ -918,8 +918,9 @@ private TExecPlanFragmentParams streamLoadPutImpl(TStreamLoadPutRequest request) @Override public TStatus snapshotLoaderReport(TSnapshotLoaderReportRequest request) throws TException { - if (Catalog.getCurrentCatalog().getBackupHandler().report(request.getTaskType(), request.getJobId(), - request.getTaskId(), request.getFinishedNum(), request.getTotalNum())) { + if (Catalog.getCurrentCatalog().getBackupHandler() + .report(request.getTaskType(), request.getJobId(), request.getTaskId(), request.getFinishedNum(), + request.getTotalNum())) { return new TStatus(TStatusCode.OK); } return new TStatus(TStatusCode.CANCELLED); diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/AlterReplicaTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/AlterReplicaTask.java index 4235ce0ded3ad25..d9ac6f0bb4743c8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/AlterReplicaTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/AlterReplicaTask.java @@ -21,12 +21,15 @@ import org.apache.doris.analysis.DescriptorTable; import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.SlotRef; +import org.apache.doris.catalog.Column; import org.apache.doris.thrift.TAlterMaterializedViewParam; import org.apache.doris.thrift.TAlterTabletReqV2; +import org.apache.doris.thrift.TColumn; import org.apache.doris.thrift.TTaskType; import com.google.common.collect.Lists; +import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -48,6 +51,7 @@ public class AlterReplicaTask extends AgentTask { private Map defineExprs; private DescriptorTable descTable; + private List baseSchemaColumns; /** * AlterReplicaTask constructor. @@ -56,7 +60,7 @@ public class AlterReplicaTask extends AgentTask { public AlterReplicaTask(long backendId, long dbId, long tableId, long partitionId, long rollupIndexId, long baseIndexId, long rollupTabletId, long baseTabletId, long newReplicaId, int newSchemaHash, int baseSchemaHash, long version, long jobId, AlterJobV2.JobType jobType, Map defineExprs, - DescriptorTable descTable) { + DescriptorTable descTable, List baseSchemaColumns) { super(null, backendId, TTaskType.ALTER, dbId, tableId, partitionId, rollupIndexId, rollupTabletId); this.baseTabletId = baseTabletId; @@ -71,6 +75,7 @@ public AlterReplicaTask(long backendId, long dbId, long tableId, long partitionI this.jobType = jobType; this.defineExprs = defineExprs; this.descTable = descTable; + this.baseSchemaColumns = baseSchemaColumns; } public long getBaseTabletId() { @@ -115,6 +120,14 @@ public TAlterTabletReqV2 toThrift() { } } req.setDescTbl(descTable.toThrift()); + + if (baseSchemaColumns != null) { + List columns = new ArrayList(); + for (Column column : baseSchemaColumns) { + columns.add(column.toThrift()); + } + req.setColumns(columns); + } return req; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/PushTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/PushTask.java index 1c32c9680b2db97..fb61fe356d3a658 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/PushTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/PushTask.java @@ -26,6 +26,7 @@ import org.apache.doris.analysis.SlotRef; import org.apache.doris.common.MarkedCountDownLatch; import org.apache.doris.thrift.TBrokerScanRange; +import org.apache.doris.thrift.TColumn; import org.apache.doris.thrift.TCondition; import org.apache.doris.thrift.TDescriptorTable; import org.apache.doris.thrift.TPriority; @@ -69,11 +70,13 @@ public class PushTask extends AgentTask { private TBrokerScanRange tBrokerScanRange; private TDescriptorTable tDescriptorTable; - public PushTask(TResourceInfo resourceInfo, long backendId, long dbId, long tableId, long partitionId, - long indexId, long tabletId, long replicaId, int schemaHash, long version, - String filePath, long fileSize, int timeoutSecond, long loadJobId, TPushType pushType, - List conditions, boolean needDecompress, TPriority priority, TTaskType taskType, - long transactionId, long signature) { + // for light schema change + private List columnsDesc = null; + + public PushTask(TResourceInfo resourceInfo, long backendId, long dbId, long tableId, long partitionId, long indexId, + long tabletId, long replicaId, int schemaHash, long version, String filePath, long fileSize, + int timeoutSecond, long loadJobId, TPushType pushType, List conditions, boolean needDecompress, + TPriority priority, TTaskType taskType, long transactionId, long signature, List columnsDesc) { super(resourceInfo, backendId, taskType, dbId, tableId, partitionId, indexId, tabletId, signature); this.replicaId = replicaId; this.schemaHash = schemaHash; @@ -92,17 +95,17 @@ public PushTask(TResourceInfo resourceInfo, long backendId, long dbId, long tabl this.transactionId = transactionId; this.tBrokerScanRange = null; this.tDescriptorTable = null; + this.columnsDesc = columnsDesc; } // for load v2 (SparkLoadJob) public PushTask(long backendId, long dbId, long tableId, long partitionId, long indexId, long tabletId, - long replicaId, int schemaHash, int timeoutSecond, long loadJobId, TPushType pushType, - TPriority priority, long transactionId, long signature, - TBrokerScanRange tBrokerScanRange, TDescriptorTable tDescriptorTable) { - this(null, backendId, dbId, tableId, partitionId, indexId, - tabletId, replicaId, schemaHash, -1, null, - 0, timeoutSecond, loadJobId, pushType, null, false, - priority, TTaskType.REALTIME_PUSH, transactionId, signature); + long replicaId, int schemaHash, int timeoutSecond, long loadJobId, TPushType pushType, TPriority priority, + long transactionId, long signature, TBrokerScanRange tBrokerScanRange, TDescriptorTable tDescriptorTable, + List columnsDesc) { + this(null, backendId, dbId, tableId, partitionId, indexId, tabletId, replicaId, schemaHash, -1, null, 0, + timeoutSecond, loadJobId, pushType, null, false, priority, TTaskType.REALTIME_PUSH, transactionId, + signature, columnsDesc); this.tBrokerScanRange = tBrokerScanRange; this.tDescriptorTable = tDescriptorTable; } @@ -171,6 +174,7 @@ public TPushReq toThrift() { LOG.warn("unknown push type. type: " + pushType.name()); break; } + request.setColumnsDesc(columnsDesc); return request; } @@ -182,8 +186,8 @@ public void setCountDownLatch(MarkedCountDownLatch latch) { public void countDownLatch(long backendId, long tabletId) { if (this.latch != null) { if (latch.markedCountDown(backendId, tabletId)) { - LOG.debug("pushTask current latch count: {}. backend: {}, tablet:{}", - latch.getCount(), backendId, tabletId); + LOG.debug("pushTask current latch count: {}. backend: {}, tablet:{}", latch.getCount(), backendId, + tabletId); } } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java index 48973f1871228f1..9fcc04dc9846c08 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java @@ -17,24 +17,341 @@ package org.apache.doris.alter; +import org.apache.doris.analysis.AlterTableStmt; import org.apache.doris.analysis.ColumnPosition; +import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Database; import org.apache.doris.catalog.KeysType; +import org.apache.doris.catalog.MaterializedIndexMeta; import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Table; +import org.apache.doris.common.FeConstants; import org.apache.doris.common.jmockit.Deencapsulation; +import org.apache.doris.utframe.TestWithFeService; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import mockit.Expectations; import mockit.Injectable; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; -public class SchemaChangeHandlerTest { +import java.util.Map; + +public class SchemaChangeHandlerTest extends TestWithFeService { + private static final Logger LOG = LogManager.getLogger(SchemaChangeHandlerTest.class); + private int jobSize = 0; + + @Override + protected void runBeforeAll() throws Exception { + FeConstants.default_scheduler_interval_millisecond = 10; + //create database db1 + createDatabase("test"); + + //create tables + String createAggTblStmtStr = "CREATE TABLE IF NOT EXISTS test.sc_agg (\n" + "user_id LARGEINT NOT NULL,\n" + + "date DATE NOT NULL,\n" + "city VARCHAR(20),\n" + "age SMALLINT,\n" + "sex TINYINT,\n" + + "last_visit_date DATETIME REPLACE DEFAULT '1970-01-01 00:00:00',\n" + "cost BIGINT SUM DEFAULT '0',\n" + + "max_dwell_time INT MAX DEFAULT '0',\n" + "min_dwell_time INT MIN DEFAULT '99999')\n" + + "AGGREGATE KEY(user_id, date, city, age, sex)\n" + "DISTRIBUTED BY HASH(user_id) BUCKETS 1\n" + + "PROPERTIES ('replication_num' = '1');"; + createTable(createAggTblStmtStr); + + String createUniqTblStmtStr = "CREATE TABLE IF NOT EXISTS test.sc_uniq (\n" + "user_id LARGEINT NOT NULL,\n" + + "username VARCHAR(50) NOT NULL,\n" + "city VARCHAR(20),\n" + "age SMALLINT,\n" + "sex TINYINT,\n" + + "phone LARGEINT,\n" + "address VARCHAR(500),\n" + "register_time DATETIME)\n" + + "UNIQUE KEY(user_id, username)\n" + "DISTRIBUTED BY HASH(user_id) BUCKETS 1\n" + + "PROPERTIES ('replication_num' = '1');"; + createTable(createUniqTblStmtStr); + + String createDupTblStmtStr = "CREATE TABLE IF NOT EXISTS test.sc_dup (\n" + "timestamp DATETIME,\n" + + "type INT,\n" + "error_code INT,\n" + "error_msg VARCHAR(1024),\n" + "op_id BIGINT,\n" + + "op_time DATETIME)\n" + "DUPLICATE KEY(timestamp, type)\n" + "DISTRIBUTED BY HASH(type) BUCKETS 1\n" + + "PROPERTIES ('replication_num' = '1');"; + createTable(createDupTblStmtStr); + } + + private void waitAlterJobDone(Map alterJobs) throws Exception { + for (AlterJobV2 alterJobV2 : alterJobs.values()) { + while (!alterJobV2.getJobState().isFinalState()) { + LOG.info("alter job {} is running. state: {}", alterJobV2.getJobId(), alterJobV2.getJobState()); + Thread.sleep(1000); + } + LOG.info("alter job {} is done. state: {}", alterJobV2.getJobId(), alterJobV2.getJobState()); + Assert.assertEquals(AlterJobV2.JobState.FINISHED, alterJobV2.getJobState()); + + Database db = Catalog.getCurrentInternalCatalog().getDbOrMetaException(alterJobV2.getDbId()); + OlapTable tbl = (OlapTable) db.getTableOrMetaException(alterJobV2.getTableId(), Table.TableType.OLAP); + while (tbl.getState() != OlapTable.OlapTableState.NORMAL) { + Thread.sleep(1000); + } + } + } + + @Test + public void testAggAddOrDropColumn() throws Exception { + LOG.info("dbName: {}", Catalog.getCurrentInternalCatalog().getDbNames()); + + Database db = Catalog.getCurrentInternalCatalog().getDbOrMetaException("default_cluster:test"); + OlapTable tbl = (OlapTable) db.getTableOrMetaException("sc_agg", Table.TableType.OLAP); + tbl.readLock(); + try { + Assertions.assertNotNull(tbl); + System.out.println(tbl.getName()); + Assertions.assertEquals("Doris", tbl.getEngine()); + Assertions.assertEquals(9, tbl.getBaseSchema().size()); + } finally { + tbl.readUnlock(); + } + + //process agg add value column schema change + String addValColStmtStr = "alter table test.sc_agg add column new_v1 int MAX default '0'"; + AlterTableStmt addValColStmt = (AlterTableStmt) parseAndAnalyzeStmt(addValColStmtStr); + Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(addValColStmt); + jobSize++; + //check alter job, do not create job + Map alterJobs = Catalog.getCurrentCatalog().getSchemaChangeHandler().getAlterJobsV2(); + Assertions.assertEquals(jobSize, alterJobs.size()); + + tbl.readLock(); + try { + Assertions.assertEquals(10, tbl.getBaseSchema().size()); + String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId()); + Assertions.assertEquals(baseIndexName, tbl.getName()); + MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId()); + Assertions.assertNotNull(indexMeta); + } finally { + tbl.readUnlock(); + } + + //process agg add key column schema change + String addKeyColStmtStr = "alter table test.sc_agg add column new_k1 int default '1'"; + AlterTableStmt addKeyColStmt = (AlterTableStmt) parseAndAnalyzeStmt(addKeyColStmtStr); + Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(addKeyColStmt); + + //check alter job + jobSize++; + Assertions.assertEquals(jobSize, alterJobs.size()); + waitAlterJobDone(alterJobs); + + tbl.readLock(); + try { + Assertions.assertEquals(11, tbl.getBaseSchema().size()); + String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId()); + Assertions.assertEquals(baseIndexName, tbl.getName()); + MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId()); + Assertions.assertNotNull(indexMeta); + } finally { + tbl.readUnlock(); + } + + //process agg drop value column schema change + String dropValColStmtStr = "alter table test.sc_agg drop column new_v1"; + AlterTableStmt dropValColStmt = (AlterTableStmt) parseAndAnalyzeStmt(dropValColStmtStr); + Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(dropValColStmt); + jobSize++; + //check alter job, do not create job + LOG.info("alterJobs:{}", alterJobs); + Assertions.assertEquals(jobSize, alterJobs.size()); + + tbl.readLock(); + try { + Assertions.assertEquals(10, tbl.getBaseSchema().size()); + String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId()); + Assertions.assertEquals(baseIndexName, tbl.getName()); + MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId()); + Assertions.assertNotNull(indexMeta); + } finally { + tbl.readUnlock(); + } + + try { + //process agg drop key column with replace schema change, expect exception. + String dropKeyColStmtStr = "alter table test.sc_agg drop column new_k1"; + AlterTableStmt dropKeyColStmt = (AlterTableStmt) parseAndAnalyzeStmt(dropKeyColStmtStr); + Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(dropKeyColStmt); + Assert.fail(); + } catch (Exception e) { + LOG.info(e.getMessage()); + } + + LOG.info("getIndexIdToSchema 1: {}", tbl.getIndexIdToSchema(true)); + + String addRollUpStmtStr = "alter table test.sc_agg add rollup agg_rollup(user_id, max_dwell_time);"; + AlterTableStmt addRollUpStmt = (AlterTableStmt) parseAndAnalyzeStmt(addRollUpStmtStr); + Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(addRollUpStmt); + // 2. check alter job + Map materializedViewAlterJobs = Catalog.getCurrentCatalog().getMaterializedViewHandler() + .getAlterJobsV2(); + waitAlterJobDone(materializedViewAlterJobs); + Assertions.assertEquals(1, materializedViewAlterJobs.size()); + + LOG.info("getIndexIdToSchema 2: {}", tbl.getIndexIdToSchema(true)); + + //process agg drop value column with rollup schema change + String dropRollUpValColStmtStr = "alter table test.sc_agg drop column max_dwell_time"; + AlterTableStmt dropRollUpValColStmt = (AlterTableStmt) parseAndAnalyzeStmt(dropRollUpValColStmtStr); + Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(dropRollUpValColStmt); + jobSize++; + //check alter job, need create job + LOG.info("alterJobs:{}", alterJobs); + Assertions.assertEquals(jobSize, alterJobs.size()); + waitAlterJobDone(materializedViewAlterJobs); + + tbl.readLock(); + try { + Assertions.assertEquals(9, tbl.getBaseSchema().size()); + String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId()); + Assertions.assertEquals(baseIndexName, tbl.getName()); + MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId()); + Assertions.assertNotNull(indexMeta); + } finally { + tbl.readUnlock(); + } + + //process agg add mul value column schema change + String addMultiValColStmtStr + = "alter table test.sc_agg add column new_v2 int MAX default '0', add column new_v3 int MAX default '1';"; + AlterTableStmt addMultiValColStmt = (AlterTableStmt) parseAndAnalyzeStmt(addMultiValColStmtStr); + Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(addMultiValColStmt); + jobSize++; + //check alter job, do not create job + Assertions.assertEquals(jobSize, alterJobs.size()); + + tbl.readLock(); + try { + Assertions.assertEquals(11, tbl.getBaseSchema().size()); + String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId()); + Assertions.assertEquals(baseIndexName, tbl.getName()); + MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId()); + Assertions.assertNotNull(indexMeta); + Assertions.assertEquals(12, tbl.getMaxColUniqueId()); + } finally { + tbl.readUnlock(); + } + } + + @Test + public void testUniqAddOrDropColumn() throws Exception { + + LOG.info("dbName: {}", Catalog.getCurrentInternalCatalog().getDbNames()); + + Database db = Catalog.getCurrentInternalCatalog().getDbOrMetaException("default_cluster:test"); + OlapTable tbl = (OlapTable) db.getTableOrMetaException("sc_uniq", Table.TableType.OLAP); + tbl.readLock(); + try { + Assertions.assertNotNull(tbl); + System.out.println(tbl.getName()); + Assertions.assertEquals("Doris", tbl.getEngine()); + Assertions.assertEquals(8, tbl.getBaseSchema().size()); + } finally { + tbl.readUnlock(); + } + + //process uniq add value column schema change + String addValColStmtStr = "alter table test.sc_uniq add column new_v1 int default '0'"; + AlterTableStmt addValColStmt = (AlterTableStmt) parseAndAnalyzeStmt(addValColStmtStr); + Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(addValColStmt); + jobSize++; + //check alter job, do not create job + Map alterJobs = Catalog.getCurrentCatalog().getSchemaChangeHandler().getAlterJobsV2(); + LOG.info("alterJobs:{}", alterJobs); + Assertions.assertEquals(jobSize, alterJobs.size()); + + tbl.readLock(); + try { + Assertions.assertEquals(9, tbl.getBaseSchema().size()); + String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId()); + Assertions.assertEquals(baseIndexName, tbl.getName()); + MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId()); + Assertions.assertNotNull(indexMeta); + } finally { + tbl.readUnlock(); + } + + //process uniq drop val column schema change + String dropValColStmtStr = "alter table test.sc_uniq drop column new_v1"; + AlterTableStmt dropValColStm = (AlterTableStmt) parseAndAnalyzeStmt(dropValColStmtStr); + Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(dropValColStm); + jobSize++; + //check alter job + Assertions.assertEquals(jobSize, alterJobs.size()); + tbl.readLock(); + try { + Assertions.assertEquals(8, tbl.getBaseSchema().size()); + String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId()); + Assertions.assertEquals(baseIndexName, tbl.getName()); + MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId()); + Assertions.assertNotNull(indexMeta); + } finally { + tbl.readUnlock(); + } + } + + @Test + public void testDupAddOrDropColumn() throws Exception { + + LOG.info("dbName: {}", Catalog.getCurrentInternalCatalog().getDbNames()); + + Database db = Catalog.getCurrentInternalCatalog().getDbOrMetaException("default_cluster:test"); + OlapTable tbl = (OlapTable) db.getTableOrMetaException("sc_dup", Table.TableType.OLAP); + tbl.readLock(); + try { + Assertions.assertNotNull(tbl); + System.out.println(tbl.getName()); + Assertions.assertEquals("Doris", tbl.getEngine()); + Assertions.assertEquals(6, tbl.getBaseSchema().size()); + } finally { + tbl.readUnlock(); + } + + //process uniq add value column schema change + String addValColStmtStr = "alter table test.sc_dup add column new_v1 int default '0'"; + AlterTableStmt addValColStmt = (AlterTableStmt) parseAndAnalyzeStmt(addValColStmtStr); + Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(addValColStmt); + jobSize++; + //check alter job, do not create job + Map alterJobs = Catalog.getCurrentCatalog().getSchemaChangeHandler().getAlterJobsV2(); + LOG.info("alterJobs:{}", alterJobs); + Assertions.assertEquals(jobSize, alterJobs.size()); + + tbl.readLock(); + try { + Assertions.assertEquals(7, tbl.getBaseSchema().size()); + String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId()); + Assertions.assertEquals(baseIndexName, tbl.getName()); + MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId()); + Assertions.assertNotNull(indexMeta); + } finally { + tbl.readUnlock(); + } + + //process uniq drop val column schema change + String dropValColStmtStr = "alter table test.sc_dup drop column new_v1"; + AlterTableStmt dropValColStm = (AlterTableStmt) parseAndAnalyzeStmt(dropValColStmtStr); + Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(dropValColStm); + jobSize++; + //check alter job + Assertions.assertEquals(jobSize, alterJobs.size()); + tbl.readLock(); + try { + Assertions.assertEquals(6, tbl.getBaseSchema().size()); + String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId()); + Assertions.assertEquals(baseIndexName, tbl.getName()); + MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId()); + Assertions.assertNotNull(indexMeta); + } finally { + tbl.readUnlock(); + } + } @Test public void testAddValueColumnOnAggMV(@Injectable OlapTable olapTable, @Injectable Column newColumn, - @Injectable ColumnPosition columnPosition) { + @Injectable ColumnPosition columnPosition) { SchemaChangeHandler schemaChangeHandler = new SchemaChangeHandler(); new Expectations() { { @@ -51,8 +368,7 @@ public void testAddValueColumnOnAggMV(@Injectable OlapTable olapTable, @Injectab try { Deencapsulation.invoke(schemaChangeHandler, "addColumnInternal", olapTable, newColumn, columnPosition, - new Long(2), new Long(1), - Maps.newHashMap(), Sets.newHashSet()); + new Long(2), new Long(1), Maps.newHashMap(), Sets.newHashSet(), false); Assert.fail(); } catch (Exception e) { System.out.println(e.getMessage()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/persist/TableAddOrDropColumnsInfoTest.java b/fe/fe-core/src/test/java/org/apache/doris/persist/TableAddOrDropColumnsInfoTest.java new file mode 100644 index 000000000000000..a97dd27bf90ecf8 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/persist/TableAddOrDropColumnsInfoTest.java @@ -0,0 +1,97 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.persist; + +import org.apache.doris.analysis.IndexDef; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Index; +import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.io.Text; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.common.collect.Lists; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +public class TableAddOrDropColumnsInfoTest { + private static String fileName = "./TableAddOrDropColumnsInfoTest"; + + @Test + public void testSerialization() throws IOException { + // 1. Write objects to file + File file = new File(fileName); + file.createNewFile(); + + DataOutputStream out = new DataOutputStream(new FileOutputStream(file)); + + long dbId = 12345678; + long tableId = 87654321; + long jobId = 23456781; + LinkedList fullSchema = new LinkedList<>(); + fullSchema.add(new Column("testCol1", ScalarType.createType(PrimitiveType.INT))); + fullSchema.add(new Column("testCol2", ScalarType.createType(PrimitiveType.VARCHAR))); + fullSchema.add(new Column("testCol3", ScalarType.createType(PrimitiveType.DATE))); + fullSchema.add(new Column("testCol4", ScalarType.createType(PrimitiveType.DATETIME))); + + Map> indexSchemaMap = new HashMap<>(); + indexSchemaMap.put(tableId, fullSchema); + + List indexes = Lists.newArrayList( + new Index("index", Lists.newArrayList("testCol1"), IndexDef.IndexType.BITMAP, "xxxxxx")); + + TableAddOrDropColumnsInfo tableAddOrDropColumnsInfo1 = new TableAddOrDropColumnsInfo(dbId, tableId, + indexSchemaMap, indexes, jobId); + + String c1Json = GsonUtils.GSON.toJson(tableAddOrDropColumnsInfo1); + Text.writeString(out, c1Json); + out.flush(); + out.close(); + + // 2. Read objects from file + DataInputStream in = new DataInputStream(new FileInputStream(file)); + + String readJson = Text.readString(in); + TableAddOrDropColumnsInfo tableAddOrDropColumnsInfo2 = GsonUtils.GSON.fromJson(readJson, + TableAddOrDropColumnsInfo.class); + + Assert.assertEquals(tableAddOrDropColumnsInfo1.getDbId(), tableAddOrDropColumnsInfo2.getDbId()); + Assert.assertEquals(tableAddOrDropColumnsInfo1.getTableId(), tableAddOrDropColumnsInfo2.getTableId()); + Assert.assertEquals(tableAddOrDropColumnsInfo1.getIndexSchemaMap(), + tableAddOrDropColumnsInfo2.getIndexSchemaMap()); + + } + + @After + public void tearDown() { + File file = new File(fileName); + file.delete(); + } +} diff --git a/gensrc/proto/descriptors.proto b/gensrc/proto/descriptors.proto index 3dab3f0beec82bc..acde58bbfa54f3b 100644 --- a/gensrc/proto/descriptors.proto +++ b/gensrc/proto/descriptors.proto @@ -21,6 +21,7 @@ package doris; option java_package = "org.apache.doris.proto"; import "types.proto"; +import "olap_file.proto"; message PSlotDescriptor { required int32 id = 1; @@ -47,6 +48,7 @@ message POlapTableIndexSchema { required int64 id = 1; repeated string columns = 2; required int32 schema_hash = 3; + repeated ColumnPB columns_desc = 4; }; message POlapTableSchemaParam { diff --git a/gensrc/proto/olap_file.proto b/gensrc/proto/olap_file.proto index 0d484a292d349c0..0758599757adbf5 100644 --- a/gensrc/proto/olap_file.proto +++ b/gensrc/proto/olap_file.proto @@ -99,6 +99,8 @@ message RowsetMetaPB { optional int64 oldest_write_timestamp = 25 [default = -1]; // latest write time optional int64 newest_write_timestamp = 26 [default = -1]; + // tablet meta pb, for compaction + optional TabletSchemaPB tablet_schema = 27; // spare field id for future use optional AlphaRowsetExtraMetaPB alpha_rowset_extra_meta_pb = 50; // to indicate whether the data between the segments overlap @@ -182,6 +184,7 @@ message ColumnPB { optional bool visible = 16 [default=true]; repeated ColumnPB children_columns = 17; repeated string children_column_names = 18; + optional int32 col_unique_id = 19 [default=-1]; // default is important } enum SortType { @@ -203,6 +206,7 @@ message TabletSchemaPB { optional SortType sort_type = 11; optional int32 sort_col_num = 12; optional segment_v2.CompressionTypePB compression_type = 13 [default=LZ4F]; + optional int32 schema_version = 14; } enum TabletStatePB { diff --git a/gensrc/thrift/AgentService.thrift b/gensrc/thrift/AgentService.thrift index 4e420de9dfc6204..78c1fe0c3bb8b75 100644 --- a/gensrc/thrift/AgentService.thrift +++ b/gensrc/thrift/AgentService.thrift @@ -25,25 +25,12 @@ include "PlanNodes.thrift" include "Descriptors.thrift" include "Exprs.thrift" -struct TColumn { - 1: required string column_name - 2: required Types.TColumnType column_type - 3: optional Types.TAggregationType aggregation_type - 4: optional bool is_key - 5: optional bool is_allow_null - 6: optional string default_value - 7: optional bool is_bloom_filter_column - 8: optional Exprs.TExpr define_expr - 9: optional bool visible = true - 10: optional list children_column -} - struct TTabletSchema { 1: required i16 short_key_column_count 2: required Types.TSchemaHash schema_hash 3: required Types.TKeysType keys_type 4: required Types.TStorageType storage_type - 5: required list columns + 5: required list columns 6: optional double bloom_filter_fpp 7: optional list indexes 8: optional bool is_in_memory @@ -168,6 +155,7 @@ struct TAlterTabletReqV2 { 7: optional list materialized_view_params 8: optional TAlterTabletType alter_tablet_type = TAlterTabletType.SCHEMA_CHANGE 9: optional Descriptors.TDescriptorTable desc_tbl + 10: optional list columns } struct TAlterMaterializedViewParam { @@ -209,6 +197,7 @@ struct TPushReq { // 14 and 15 are used by spark load 14: optional PlanNodes.TBrokerScanRange broker_scan_range 15: optional Descriptors.TDescriptorTable desc_tbl + 16: optional list columns_desc } struct TCloneReq { diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift index 187df7e0b3bfcf3..787a6e606e50398 100644 --- a/gensrc/thrift/Descriptors.thrift +++ b/gensrc/thrift/Descriptors.thrift @@ -21,6 +21,21 @@ namespace java org.apache.doris.thrift include "Types.thrift" include "Exprs.thrift" +struct TColumn { + 1: required string column_name + 2: required Types.TColumnType column_type + 3: optional Types.TAggregationType aggregation_type + 4: optional bool is_key + 5: optional bool is_allow_null + 6: optional string default_value + 7: optional bool is_bloom_filter_column + 8: optional Exprs.TExpr define_expr + 9: optional bool visible = true + 10: optional list children_column + 11: optional i32 col_unique_id = -1 + 12: optional bool has_bitmap_index = false +} + struct TSlotDescriptor { 1: required Types.TSlotId id 2: required Types.TTupleId parent @@ -32,6 +47,7 @@ struct TSlotDescriptor { 8: required string colName; 9: required i32 slotIdx 10: required bool isMaterialized + 11: optional i32 col_unique_id = -1 } struct TTupleDescriptor { @@ -158,6 +174,7 @@ struct TOlapTableIndexSchema { 1: required i64 id 2: required list columns 3: required i32 schema_hash + 4: required list columns_desc } struct TOlapTableSchemaParam { diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index dec3ca2f64da17e..2469ebd4e29a0b3 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -419,6 +419,7 @@ struct TOlapScanNode { 5: optional string sort_column 6: optional Types.TKeysType keyType 7: optional string table_name + 8: required list columns_desc } struct TEqJoinCondition { diff --git a/regression-test/data/schema_change/test_compaction_schema_change.out b/regression-test/data/schema_change/test_compaction_schema_change.out new file mode 100644 index 000000000000000..c7f14cd6587e582 --- /dev/null +++ b/regression-test/data/schema_change/test_compaction_schema_change.out @@ -0,0 +1,11 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !compaction -- +100 + +-- !compaction -- +100 + +-- !compaction -- +1 +2 + diff --git a/regression-test/data/schema_change/test_delete_schema_change.out b/regression-test/data/schema_change/test_delete_schema_change.out new file mode 100644 index 000000000000000..ff8a11901d0df8a --- /dev/null +++ b/regression-test/data/schema_change/test_delete_schema_change.out @@ -0,0 +1,44 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !test_delete_schema_change -- +0 + +-- !test_delete_schema_change_2 -- +0 + +-- !test_delete_schema_change_3 -- +1 + +-- !test_delete_schema_change_4 -- +1 + +-- !test_delete_schema_change_5 -- +1 2017-10-01 Beijing 10 1 2020-01-01T00:00 2020-01-01T00:00 2020-01-01T00:00 1 30 20 +2 2017-10-01 Beijing 10 1 2020-01-02T00:00 2020-01-02T00:00 2020-01-02T00:00 1 31 21 + +-- !test_delete_schema_change_6 -- +0 + +-- !test_delete_schema_change_7 -- +2 2017-10-01 Beijing 10 1 2020-01-02T00:00 2020-01-02T00:00 2020-01-02T00:00 1 31 21 1 +1 2017-10-01 Beijing 10 1 2020-01-01T00:00 2020-01-01T00:00 2020-01-01T00:00 1 30 20 1 + +-- !test_delete_schema_change_8 -- +1 + +-- !test_delete_schema_change_9 -- +1 + +-- !test_delete_schema_change_10 -- +1 + +-- !test_delete_schema_change_11 -- +1 + +-- !test_delete_schema_change_12 -- +0 + +-- !test_delete_schema_change_13 -- +3 2017-10-01 Beijing 10 1 2020-01-03T00:00 2020-01-03T00:00 2020-01-03T00:00 1 32 20 2 +2 2017-10-01 Beijing 10 1 2020-01-03T00:00 2020-01-03T00:00 2020-01-03T00:00 1 32 20 2 +1 2017-10-01 Beijing 10 1 2020-01-02T00:00 2020-01-02T00:00 2020-01-02T00:00 1 31 19 2 + diff --git a/regression-test/data/schema_change/test_update_schema_change.out b/regression-test/data/schema_change/test_update_schema_change.out new file mode 100644 index 000000000000000..aeacfd28c66222d --- /dev/null +++ b/regression-test/data/schema_change/test_update_schema_change.out @@ -0,0 +1,64 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !test_update_schema_change -- +0 + +-- !test_update_schema_change_2 -- +0 + +-- !test_update_schema_change_3 -- +1 + +-- !test_update_schema_change_4 -- +1 + +-- !test_update_schema_change_5 -- +1 2017-10-01 Beijing 10 1 2020-01-01T00:00 2020-01-01T00:00 2020-01-01T00:00 1 30 20 +2 2017-10-01 Beijing 10 1 2020-01-02T00:00 2020-01-02T00:00 2020-01-02T00:00 1 31 21 + +-- !test_update_schema_change_6 -- +0 + +-- !test_update_schema_change_7 -- +2 2017-10-01 Beijing 10 1 2020-01-02T00:00 2020-01-02T00:00 2020-01-02T00:00 1 31 21 1 +1 2017-10-01 Beijing 10 1 2020-01-01T00:00 2020-01-01T00:00 2020-01-01T00:00 1 30 20 1 + +-- !test_update_schema_change_8 -- +1 + +-- !test_update_schema_change_9 -- +1 2017-10-01 Beijing 10 1 2020-01-01T00:00 2020-01-01T00:00 2020-01-01T00:00 1 30 20 2 +2 2017-10-01 Beijing 10 1 2020-01-02T00:00 2020-01-02T00:00 2020-01-02T00:00 1 31 21 1 + +-- !test_update_schema_change_10 -- +1 + +-- !test_update_schema_change_11 -- +1 + +-- !test_update_schema_change_12 -- +2 + +-- !test_update_schema_change_13 -- +5 2017-10-01 Beijing 10 1 2020-01-02T00:00 2020-01-02T00:00 2020-01-02T00:00 1 31 21 20 +3 2017-10-01 Beijing 10 1 2020-01-01T00:00 2020-01-01T00:00 2020-01-01T00:00 1 30 20 20 +2 2017-10-01 Beijing 10 1 2020-01-02T00:00 2020-01-02T00:00 2020-01-02T00:00 1 31 21 1 +1 2017-10-01 Beijing 10 1 2020-01-01T00:00 2020-01-01T00:00 2020-01-01T00:00 1 30 20 20 + +-- !test_update_schema_change_14 -- +0 + +-- !test_update_schema_change_15 -- +5 2017-10-01 Beijing 10 1 2020-01-02T00:00 2020-01-02T00:00 2020-01-02T00:00 1 31 21 +3 2017-10-01 Beijing 10 1 2020-01-01T00:00 2020-01-01T00:00 2020-01-01T00:00 1 30 20 +2 2017-10-01 Beijing 10 1 2020-01-02T00:00 2020-01-02T00:00 2020-01-02T00:00 1 31 21 +1 2017-10-01 Beijing 10 1 2020-01-01T00:00 2020-01-01T00:00 2020-01-01T00:00 1 30 20 + +-- !test_update_schema_change_16 -- +1 + +-- !test_update_schema_change_17 -- +5 2017-10-01 Beijing 10 1 2020-01-02T00:00 2020-01-02T00:00 2020-01-02T00:00 20 31 21 +3 2017-10-01 Beijing 10 1 2020-01-01T00:00 2020-01-01T00:00 2020-01-01T00:00 1 30 20 +2 2017-10-01 Beijing 10 1 2020-01-02T00:00 2020-01-02T00:00 2020-01-02T00:00 1 31 21 +1 2017-10-01 Beijing 10 1 2020-01-01T00:00 2020-01-01T00:00 2020-01-01T00:00 1 30 20 + diff --git a/regression-test/suites/schema_change/test_agg_keys_schema_change.groovy b/regression-test/suites/schema_change/test_agg_keys_schema_change.groovy new file mode 100644 index 000000000000000..96308a9b6d30bfa --- /dev/null +++ b/regression-test/suites/schema_change/test_agg_keys_schema_change.groovy @@ -0,0 +1,230 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite ("test_agg_keys_schema_change") { + def tableName = "schema_change_agg_keys_regression_test" + + try { + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ + CREATE TABLE ${tableName} ( + `user_id` LARGEINT NOT NULL COMMENT "用户id", + `date` DATE NOT NULL COMMENT "数据灌入日期时间", + `city` VARCHAR(20) COMMENT "用户所在城市", + `age` SMALLINT COMMENT "用户年龄", + `sex` TINYINT COMMENT "用户性别", + + `cost` BIGINT SUM DEFAULT "0" COMMENT "用户总消费", + `max_dwell_time` INT MAX DEFAULT "0" COMMENT "用户最大停留时间", + `min_dwell_time` INT MIN DEFAULT "99999" COMMENT "用户最小停留时间", + `hll_col` HLL HLL_UNION NOT NULL COMMENT "HLL列", + `bitmap_col` Bitmap BITMAP_UNION NOT NULL COMMENT "bitmap列") + AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + PROPERTIES ( "replication_num" = "1" ); + """ + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, 1, 30, 20, hll_hash(1), to_bitmap(1)) + """ + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, 1, 31, 19, hll_hash(2), to_bitmap(2)) + """ + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, 1, 31, 21, hll_hash(2), to_bitmap(2)) + """ + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(3), to_bitmap(3)) + """ + + def result = "null"; + result = sql """ select * from ${tableName} order by user_id""" + assertTrue(result.size() == 2) + assertTrue(result[0].size() == 10) + assertTrue(result[0][5] == 2, "user id 1 cost should be 2") + assertTrue(result[1][5] == 2, "user id 2 cost should be 2") + + // add key column case 1, not light schema change + sql """ + ALTER table ${tableName} ADD COLUMN new_key_column INT default "2" + """ + + result = "null" + while (!result.contains("FINISHED")){ + result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;" + result = result.toString() + logger.info("result: ${result}") + if(result.contains("CANCELLED")){ + break + } + Thread.sleep(1000) + } + + sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`cost`,`max_dwell_time`,`min_dwell_time`, `hll_col`, `bitmap_col`) + VALUES + (3, '2017-10-01', 'Beijing', 10, 1, 100, 32, 20, hll_hash(4), to_bitmap(4)) + """ + result = "null" + result = sql """SELECT * FROM ${tableName} WHERE user_id = 3""" + + assertTrue(result.size() == 1) + assertTrue(result[0][5] == 2, "new key column default value should be 2") + assertTrue(result[0].size() == 11) + + // add key column case 2 + sql """ INSERT INTO ${tableName} VALUES + (3, '2017-10-01', 'Beijing', 10, 1, 3, 110, 32, 20, hll_hash(4), to_bitmap(4)) + """ + result = "null" + result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """ + + assertTrue(result.size() == 2) + assertTrue(result[0].size() == 11) + + result = "null" + result = sql """ select count(*) from ${tableName} """ + logger.info("result.size:" + result.size() + " result[0].size:" + result[0].size + " " + result[0][0]) + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 1) + assertTrue(result[0][0] == 4, "total count is 4") + + // drop key column, not light schema change + sql """ + ALTER TABLE ${tableName} DROP COLUMN new_key_column + """ + result = "null" + while (!result.contains("FINISHED")){ + result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;" + result = result.toString() + logger.info("result: ${result}") + if(result.contains("CANCELLED")){ + break + } + Thread.sleep(1000) + } + + result = sql """ select * from ${tableName} where user_id = 3 """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 10) + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + Thread.sleep(30 * 1000) + // compaction + String[][] tablets = sql """ show tablets from ${tableName}; """ + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + logger.info("run compaction:" + tablet_id) + StringBuilder sb = new StringBuilder(); + sb.append("curl -X POST http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run?tablet_id=") + sb.append(tablet_id) + sb.append("&compact_type=cumulative") + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + } + + // wait for all compactions done + for (String[] tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run_status?tablet_id=") + sb.append(tablet_id) + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + result = sql """ select count(*) from ${tableName} """ + assertTrue(result.size() == 1) + assertTrue(result[0][0] == 4) + + result = sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 10) + + int rowCount = 0 + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/show?tablet_id=") + sb.append(tablet_id) + String command = sb.toString() + // wait for cleaning stale_rowsets + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + rowCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + logger.info("size:" + rowCount) + assertTrue(rowCount <= 8) + } finally { + //try_sql("DROP TABLE IF EXISTS ${tableName}") + } +} \ No newline at end of file diff --git a/regression-test/suites/schema_change/test_agg_mv_schema_change.groovy b/regression-test/suites/schema_change/test_agg_mv_schema_change.groovy new file mode 100644 index 000000000000000..a6b104ab70df8c0 --- /dev/null +++ b/regression-test/suites/schema_change/test_agg_mv_schema_change.groovy @@ -0,0 +1,214 @@ + +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite ("test_agg_mv_schema_change") { + def tableName = "schema_change_agg_mv_regression_test" + + try { + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ + CREATE TABLE ${tableName} ( + `user_id` LARGEINT NOT NULL COMMENT "用户id", + `date` DATE NOT NULL COMMENT "数据灌入日期时间", + `city` VARCHAR(20) COMMENT "用户所在城市", + `age` SMALLINT COMMENT "用户年龄", + `sex` TINYINT COMMENT "用户性别", + + `cost` BIGINT SUM DEFAULT "0" COMMENT "用户总消费", + `max_dwell_time` INT MAX DEFAULT "0" COMMENT "用户最大停留时间", + `min_dwell_time` INT MIN DEFAULT "99999" COMMENT "用户最小停留时间", + `hll_col` HLL HLL_UNION NOT NULL COMMENT "HLL列", + `bitmap_col` Bitmap BITMAP_UNION NOT NULL COMMENT "bitmap列") + AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + PROPERTIES ( "replication_num" = "1" ); + """ + + //add materialized view + def result = "null" + def mvName = "mv1" + sql "create materialized view ${mvName} as select user_id, date, city, age, sex, sum(cost) from ${tableName} group by user_id, date, city, age, sex, cost;" + while (!result.contains("FINISHED")){ + result = sql "SHOW ALTER TABLE MATERIALIZED VIEW WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;" + result = result.toString() + logger.info("result: ${result}") + if(result.contains("CANCELLED")){ + break + } + Thread.sleep(1000) + } + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, 1, 30, 20, hll_hash(1), to_bitmap(1)) + """ + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, 1, 31, 19, hll_hash(2), to_bitmap(2)) + """ + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, 1, 31, 21, hll_hash(2), to_bitmap(2)) + """ + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(3), to_bitmap(3)) + """ + + result = "null"; + result = sql """ select * from ${tableName} """ + assertTrue(result.size() == 2) + assertTrue(result[0].size() == 10) + assertTrue(result[0][5] == 2, "user id 1 cost should be 2") + assertTrue(result[1][5] == 2, "user id 2 cost should be 2") + assertTrue(result[0].size() == 10) + + // drop value column with mv, not light schema change + sql """ + ALTER TABLE ${tableName} DROP COLUMN cost + """ + + result = "null" + while (!result.contains("FINISHED")){ + result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;" + result = result.toString() + logger.info("result: ${result}") + if(result.contains("CANCELLED")) { + log.info("rollup job is cancelled, result: ${result}".toString()) + break + } + Thread.sleep(1000) + } + + sql """ INSERT INTO ${tableName} (`user_id`, `date`, `city`, `age`, `sex`, `max_dwell_time`,`min_dwell_time`, `hll_col`, `bitmap_col`) + VALUES + (3, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(4), to_bitmap(4)) + """ + result = "null" + result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """ + + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 9) + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + Thread.sleep(30 * 1000) + // compaction + String[][] tablets = sql """ show tablets from ${tableName}; """ + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + logger.info("run compaction:" + tablet_id) + StringBuilder sb = new StringBuilder(); + sb.append("curl -X POST http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run?tablet_id=") + sb.append(tablet_id) + sb.append("&compact_type=cumulative") + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + //assertEquals(code, 0) + } + + // wait for all compactions done + for (String[] tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run_status?tablet_id=") + sb.append(tablet_id) + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + result = sql """ select count(*) from ${tableName} """ + assertTrue(result.size() == 1) + assertTrue(result[0][0] == 4) + + result = sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 9) + + int rowCount = 0 + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/show?tablet_id=") + sb.append(tablet_id) + String command = sb.toString() + // wait for cleaning stale_rowsets + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + rowCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + logger.info("size:" + rowCount) + assertTrue(rowCount <= 14) + } finally { + //try_sql("DROP TABLE IF EXISTS ${tableName}") + } + +} + + \ No newline at end of file diff --git a/regression-test/suites/schema_change/test_agg_rollup_schema_change.groovy b/regression-test/suites/schema_change/test_agg_rollup_schema_change.groovy new file mode 100644 index 000000000000000..232635b483e3cf8 --- /dev/null +++ b/regression-test/suites/schema_change/test_agg_rollup_schema_change.groovy @@ -0,0 +1,211 @@ + +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite ("test_agg_rollup_schema_change") { + def tableName = "schema_change_agg_rollup_regression_test" + + try { + sql """ DROP TABLE IF EXISTS ${tableName} """ + sql """ + CREATE TABLE ${tableName} ( + `user_id` LARGEINT NOT NULL COMMENT "用户id", + `date` DATE NOT NULL COMMENT "数据灌入日期时间", + `city` VARCHAR(20) COMMENT "用户所在城市", + `age` SMALLINT COMMENT "用户年龄", + `sex` TINYINT COMMENT "用户性别", + + `cost` BIGINT SUM DEFAULT "0" COMMENT "用户总消费", + `max_dwell_time` INT MAX DEFAULT "0" COMMENT "用户最大停留时间", + `min_dwell_time` INT MIN DEFAULT "99999" COMMENT "用户最小停留时间", + `hll_col` HLL HLL_UNION NOT NULL COMMENT "HLL列", + `bitmap_col` Bitmap BITMAP_UNION NOT NULL COMMENT "bitmap列") + AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + PROPERTIES ( "replication_num" = "1" ); + """ + + //add rollup + def result = "null" + def rollupName = "rollup_cost" + sql "ALTER TABLE ${tableName} ADD ROLLUP ${rollupName}(`user_id`,`date`,`city`,`age`,`sex`, cost);" + while (!result.contains("FINISHED")){ + result = sql "SHOW ALTER TABLE ROLLUP WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;" + result = result.toString() + logger.info("result: ${result}") + if(result.contains("CANCELLED")){ + break + } + Thread.sleep(1000) + } + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, 1, 30, 20, hll_hash(1), to_bitmap(1)) + """ + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, 1, 31, 19, hll_hash(2), to_bitmap(2)) + """ + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, 1, 31, 21, hll_hash(2), to_bitmap(2)) + """ + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(3), to_bitmap(3)) + """ + + result = "null"; + result = sql """ select * from ${tableName} """ + assertTrue(result.size() == 2) + assertTrue(result[0].size() == 10) + assertTrue(result[0][5] == 2, "user id 1 cost should be 2") + assertTrue(result[1][5] == 2, "user id 2 cost should be 2") + assertTrue(result[0].size() == 10) + + // drop value column with rollup, not light schema change + sql """ + ALTER TABLE ${tableName} DROP COLUMN cost + """ + + result = "null" + while (!result.contains("FINISHED")){ + result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;" + result = result.toString() + logger.info("result: ${result}") + if(result.contains("CANCELLED")) { + log.info("rollup job is cancelled, result: ${result}".toString()) + break + } + Thread.sleep(1000) + } + + sql """ INSERT INTO ${tableName} (`user_id`, `date`, `city`, `age`, `sex`, `max_dwell_time`,`min_dwell_time`, `hll_col`, `bitmap_col`) + VALUES + (3, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(4), to_bitmap(4)) + """ + result = "null" + result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """ + + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 9) + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5)) + """ + + Thread.sleep(30 * 1000) + // compaction + String[][] tablets = sql """ show tablets from ${tableName}; """ + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + logger.info("run compaction:" + tablet_id) + StringBuilder sb = new StringBuilder(); + sb.append("curl -X POST http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run?tablet_id=") + sb.append(tablet_id) + sb.append("&compact_type=cumulative") + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + //assertEquals(code, 0) + } + + // wait for all compactions done + for (String[] tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run_status?tablet_id=") + sb.append(tablet_id) + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + result = sql """ select count(*) from ${tableName} """ + assertTrue(result.size() == 1) + assertTrue(result[0][0] == 4) + + result = sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 9) + + int rowCount = 0 + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/show?tablet_id=") + sb.append(tablet_id) + String command = sb.toString() + // wait for cleaning stale_rowsets + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + rowCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + logger.info("size:" + rowCount) + assertTrue(rowCount <= 12) + } finally { + //try_sql("DROP TABLE IF EXISTS ${tableName}") + } +} diff --git a/regression-test/suites/schema_change/test_agg_vals_schema_change.groovy b/regression-test/suites/schema_change/test_agg_vals_schema_change.groovy new file mode 100644 index 000000000000000..d83f0e4c739abfe --- /dev/null +++ b/regression-test/suites/schema_change/test_agg_vals_schema_change.groovy @@ -0,0 +1,227 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite ("test_agg_vals_schema_change") { + def tableName = "schema_change_agg_vals_regression_test" + + try { + sql """ DROP TABLE IF EXISTS ${tableName} """ + + sql """ + CREATE TABLE ${tableName} ( + `user_id` LARGEINT NOT NULL COMMENT "用户id", + `date` DATE NOT NULL COMMENT "数据灌入日期时间", + `city` VARCHAR(20) COMMENT "用户所在城市", + `age` SMALLINT COMMENT "用户年龄", + `sex` TINYINT COMMENT "用户性别", + `last_visit_date` DATETIME REPLACE DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `last_update_date` DATETIME REPLACE DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间", + `last_visit_date_not_null` DATETIME REPLACE NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `cost` BIGINT SUM DEFAULT "0" COMMENT "用户总消费", + `max_dwell_time` INT MAX DEFAULT "0" COMMENT "用户最大停留时间", + `min_dwell_time` INT MIN DEFAULT "99999" COMMENT "用户最小停留时间", + `hll_col` HLL HLL_UNION NOT NULL COMMENT "HLL列", + `bitmap_col` Bitmap BITMAP_UNION NOT NULL COMMENT "bitmap列") + AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + PROPERTIES ( "replication_num" = "1" ); + """ + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20, hll_hash(1), to_bitmap(1)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19, hll_hash(2), to_bitmap(2)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21, hll_hash(2), to_bitmap(2)) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(3), to_bitmap(3)) + """ + def result1 = sql """ + select * from ${tableName} + """ + assertTrue(result1.size() == 2) + assertTrue(result1[0].size() == 13) + assertTrue(result1[0][8] == 2, "user id 1 cost should be 2") + assertTrue(result1[1][8] == 2, "user id 2 cost should be 2") + + // add column + sql """ + ALTER table ${tableName} ADD COLUMN new_column INT MAX default "1" + """ + + def result2 = sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + assertTrue(result1[0][8] == 2, "user id 2 cost should be 2") + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(4), to_bitmap(4), 2) + """ + result2 = sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + assertTrue(result2[0][8] == 3, "user id 2 cost should be 3") + + + sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`last_visit_date`,`last_update_date`, + `last_visit_date_not_null`,`cost`,`max_dwell_time`,`min_dwell_time`, `hll_col`, `bitmap_col`) + VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(4), to_bitmap(4)) + """ + + result2 = sql """ SELECT * FROM ${tableName} WHERE user_id=3 """ + + assertTrue(result2.size() == 1) + assertTrue(result2[0].size() == 14) + assertTrue(result2[0][13] == 1, "new add column default value should be 1") + + sql """ INSERT INTO ${tableName} VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(4), to_bitmap(4), 2) + """ + def result3 = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """ + + assertTrue(result3.size() == 1) + assertTrue(result3[0].size() == 14) + assertTrue(result3[0][13] == 2, "new add column value is set to 2") + + def result4 = sql """ select count(*) from ${tableName} """ + logger.info("result4.size:"+result4.size() + " result4[0].size:" + result4[0].size + " " + result4[0][0]) + assertTrue(result4.size() == 1) + assertTrue(result4[0].size() == 1) + assertTrue(result4[0][0] == 3, "total count is 3") + + // drop column + sql """ + ALTER TABLE ${tableName} DROP COLUMN last_visit_date + """ + def result5 = sql """ select * from ${tableName} where user_id = 3 """ + assertTrue(result5.size() == 1) + assertTrue(result5[0].size() == 13) + + sql """ INSERT INTO ${tableName} VALUES + (4, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(4), to_bitmap(4), 2) + """ + + def result6 = sql """ select * from ${tableName} where user_id = 4 """ + assertTrue(result6.size() == 1) + assertTrue(result6[0].size() == 13) + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(5), to_bitmap(5), 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(5), to_bitmap(5), 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(5), to_bitmap(5), 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(5), to_bitmap(5), 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(5), to_bitmap(5), 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(5), to_bitmap(5), 2) + """ + + Thread.sleep(30 * 1000) + // compaction + String[][] tablets = sql """ show tablets from ${tableName}; """ + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + logger.info("run compaction:" + tablet_id) + StringBuilder sb = new StringBuilder(); + sb.append("curl -X POST http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run?tablet_id=") + sb.append(tablet_id) + sb.append("&compact_type=cumulative") + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + //assertEquals(code, 0) + } + + // wait for all compactions done + for (String[] tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run_status?tablet_id=") + sb.append(tablet_id) + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + def result7 = sql """ select count(*) from ${tableName} """ + assertTrue(result7.size() == 1) + assertTrue(result7[0][0] == 5) + + def result8 = sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + assertTrue(result8.size() == 1) + assertTrue(result8[0].size() == 13) + + int rowCount = 0 + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/show?tablet_id=") + sb.append(tablet_id) + String command = sb.toString() + // wait for cleaning stale_rowsets + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + rowCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + logger.info("size:" + rowCount) + assertTrue(rowCount <= 8) + } finally { + //try_sql("DROP TABLE IF EXISTS ${tableName}") + } + +} diff --git a/regression-test/suites/schema_change/test_delete_schema_change.sql b/regression-test/suites/schema_change/test_delete_schema_change.sql new file mode 100644 index 000000000000000..e3a92a38f5a8f70 --- /dev/null +++ b/regression-test/suites/schema_change/test_delete_schema_change.sql @@ -0,0 +1,44 @@ +DROP TABLE IF EXISTS schema_change_delete_regression_test; + +CREATE TABLE schema_change_delete_regression_test ( + `user_id` LARGEINT NOT NULL COMMENT "用户id", + `date` DATE NOT NULL COMMENT "数据灌入日期时间", + `city` VARCHAR(20) COMMENT "用户所在城市", + `age` SMALLINT COMMENT "用户年龄", + `sex` TINYINT COMMENT "用户性别", + `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间", + `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `cost` BIGINT DEFAULT "0" COMMENT "用户总消费", + `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", + `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") + DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + PROPERTIES ( "replication_num" = "1" ); + +INSERT INTO schema_change_delete_regression_test VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20); + +INSERT INTO schema_change_delete_regression_test VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21); + +SELECT * FROM schema_change_delete_regression_test order by user_id ASC, last_visit_date; + +ALTER table schema_change_delete_regression_test ADD COLUMN new_column INT default "1"; + +SELECT * FROM schema_change_delete_regression_test order by user_id DESC, last_visit_date; + +INSERT INTO schema_change_delete_regression_test VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19, 2); + +INSERT INTO schema_change_delete_regression_test VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2); + +INSERT INTO schema_change_delete_regression_test VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 1); + +INSERT INTO schema_change_delete_regression_test VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2); + +DELETE FROM schema_change_delete_regression_test where new_column = 1; + +SELECT * FROM schema_change_delete_regression_test order by user_id DESC, last_visit_date; \ No newline at end of file diff --git a/regression-test/suites/schema_change/test_dup_keys_schema_change.groovy b/regression-test/suites/schema_change/test_dup_keys_schema_change.groovy new file mode 100644 index 000000000000000..22763baf2613ff2 --- /dev/null +++ b/regression-test/suites/schema_change/test_dup_keys_schema_change.groovy @@ -0,0 +1,229 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite ("test_dup_keys_schema_change") { + def tableName = "schema_change_dup_keys_regression_test" + + try { + sql """ DROP TABLE IF EXISTS ${tableName} """ + + sql """ + CREATE TABLE ${tableName} ( + `user_id` LARGEINT NOT NULL COMMENT "用户id", + `date` DATE NOT NULL COMMENT "数据灌入日期时间", + `city` VARCHAR(20) COMMENT "用户所在城市", + `age` SMALLINT COMMENT "用户年龄", + `sex` TINYINT COMMENT "用户性别", + `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间", + `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `cost` BIGINT DEFAULT "0" COMMENT "用户总消费", + `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", + `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") + DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + PROPERTIES ( "replication_num" = "1" ); + """ + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20) + """ + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20) + """ + def result = sql """ + select count(*) from ${tableName} + """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 1) + assertTrue(result[0][0] == 4, "total columns should be 4 rows") + + // add column + sql """ + ALTER table ${tableName} ADD COLUMN new_column INT default "1" + """ + + sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + + sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`last_visit_date`,`last_update_date`, + `last_visit_date_not_null`,`cost`,`max_dwell_time`,`min_dwell_time`) + VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20) + """ + + result = sql """ SELECT * FROM ${tableName} WHERE user_id=3 """ + + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 12) + assertTrue(result[0][11] == 1, "new add column default value should be 1") + + sql """ INSERT INTO ${tableName} VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """ + + assertTrue(result.size() == 2) + assertTrue(result[0].size() == 12) + assertTrue(result[1][11] == 2, "new add column value is set to 2") + + result = sql """ select count(*) from ${tableName} """ + logger.info("result.size:" + result.size() + " result[0].size:" + result[0].size + " " + result[0][0]) + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 1) + assertTrue(result[0][0] == 6, "total count is 6") + + // drop column + sql """ + ALTER TABLE ${tableName} DROP COLUMN sex + """ + result = "null" + while (!result.contains("FINISHED")){ + result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;" + result = result.toString() + logger.info("result: ${result}") + if(result.contains("CANCELLED")) { + log.info("rollup job is cancelled, result: ${result}".toString()) + break + } + Thread.sleep(1000) + } + result = sql """ select * from ${tableName} where user_id = 3 """ + assertTrue(result.size() == 2) + assertTrue(result[0].size() == 11) + + sql """ INSERT INTO ${tableName} VALUES + (4, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + + result = sql """ select * from ${tableName} where user_id = 4 """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 11) + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + + Thread.sleep(30 * 1000) + // compaction + String[][] tablets = sql """ show tablets from ${tableName}; """ + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + logger.info("run compaction:" + tablet_id) + StringBuilder sb = new StringBuilder(); + sb.append("curl -X POST http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run?tablet_id=") + sb.append(tablet_id) + sb.append("&compact_type=cumulative") + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + //assertEquals(code, 0) + } + + // wait for all compactions done + for (String[] tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run_status?tablet_id=") + sb.append(tablet_id) + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + result = sql """ select count(*) from ${tableName} """ + assertTrue(result.size() == 1) + assertTrue(result[0][0] == 13) + + + result = sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + assertTrue(result.size() == 2) + assertTrue(result[0].size() == 11) + + int rowCount = 0 + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/show?tablet_id=") + sb.append(tablet_id) + String command = sb.toString() + // wait for cleaning stale_rowsets + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + rowCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + logger.info("size:" + rowCount) + assertTrue(rowCount < 10) + } finally { + //try_sql("DROP TABLE IF EXISTS ${tableName}") + } + +} diff --git a/regression-test/suites/schema_change/test_dup_mv_schema_change.groovy b/regression-test/suites/schema_change/test_dup_mv_schema_change.groovy new file mode 100644 index 000000000000000..e290833bb4502f9 --- /dev/null +++ b/regression-test/suites/schema_change/test_dup_mv_schema_change.groovy @@ -0,0 +1,244 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite ("test_dup_mv_schema_change") { + def tableName = "schema_change_dup_mv_regression_test" + + try { + sql """ DROP TABLE IF EXISTS ${tableName} """ + + sql """ + CREATE TABLE ${tableName} ( + `user_id` LARGEINT NOT NULL COMMENT "用户id", + `date` DATE NOT NULL COMMENT "数据灌入日期时间", + `city` VARCHAR(20) COMMENT "用户所在城市", + `age` SMALLINT COMMENT "用户年龄", + `sex` TINYINT COMMENT "用户性别", + `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间", + `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `cost` BIGINT DEFAULT "0" COMMENT "用户总消费", + `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", + `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") + DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + PROPERTIES ( "replication_num" = "1" ); + """ + + //add materialized view + def result = "null" + def mvName = "mv1" + sql "create materialized view ${mvName} as select user_id, date, city, age,sex from ${tableName};" + while (!result.contains("FINISHED")){ + result = sql "SHOW ALTER TABLE MATERIALIZED VIEW WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;" + result = result.toString() + logger.info("result: ${result}") + if(result.contains("CANCELLED")){ + break + } + Thread.sleep(1000) + } + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20) + """ + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20) + """ + result = sql """ + select count(*) from ${tableName} + """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 1) + assertTrue(result[0][0] == 4, "total columns should be 4 rows") + + // add column + sql """ + ALTER table ${tableName} ADD COLUMN new_column INT default "1" + """ + + sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + + sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`last_visit_date`,`last_update_date`, + `last_visit_date_not_null`,`cost`,`max_dwell_time`,`min_dwell_time`) + VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20) + """ + + result = sql """ SELECT * FROM ${tableName} WHERE user_id=3 """ + + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 12) + assertTrue(result[0][11] == 1, "new add column default value should be 1") + + sql """ INSERT INTO ${tableName} VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """ + + assertTrue(result.size() == 2) + assertTrue(result[0].size() == 12) + assertTrue(result[1][11] == 2, "new add column value is set to 2") + + result = sql """ select count(*) from ${tableName} """ + logger.info("result.size:" + result.size() + " result[0].size:" + result[0].size + " " + result[0][0]) + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 1) + assertTrue(result[0][0] == 6, "total count is 6") + + // drop column + sql """ + ALTER TABLE ${tableName} DROP COLUMN sex + """ + result = "null" + while (!result.contains("FINISHED")){ + result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;" + result = result.toString() + logger.info("result: ${result}") + if(result.contains("CANCELLED")) { + log.info("rollup job is cancelled, result: ${result}".toString()) + break + } + Thread.sleep(1000) + } + + result = sql """ select * from ${tableName} where user_id = 3 """ + assertTrue(result.size() == 2) + assertTrue(result[0].size() == 11) + + sql """ INSERT INTO ${tableName} VALUES + (4, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + + result = sql """ select * from ${tableName} where user_id = 4 """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 11) + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + + Thread.sleep(30 * 1000) + // compaction + String[][] tablets = sql """ show tablets from ${tableName}; """ + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + logger.info("run compaction:" + tablet_id) + StringBuilder sb = new StringBuilder(); + sb.append("curl -X POST http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run?tablet_id=") + sb.append(tablet_id) + sb.append("&compact_type=cumulative") + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + //assertEquals(code, 0) + } + + // wait for all compactions done + for (String[] tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run_status?tablet_id=") + sb.append(tablet_id) + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + result = sql """ select count(*) from ${tableName} """ + assertTrue(result.size() == 1) + assertTrue(result[0][0] == 13) + + + result = sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + assertTrue(result.size() == 2) + assertTrue(result[0].size() == 11) + + int rowCount = 0 + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/show?tablet_id=") + sb.append(tablet_id) + String command = sb.toString() + // wait for cleaning stale_rowsets + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + rowCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + logger.info("size:" + rowCount) + assertTrue(rowCount <= 14) + } finally { + //try_sql("DROP TABLE IF EXISTS ${tableName}") + } + +} diff --git a/regression-test/suites/schema_change/test_dup_rollup_schema_change.groovy b/regression-test/suites/schema_change/test_dup_rollup_schema_change.groovy new file mode 100644 index 000000000000000..dc3e2fb531d0500 --- /dev/null +++ b/regression-test/suites/schema_change/test_dup_rollup_schema_change.groovy @@ -0,0 +1,244 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite ("test_dup_rollup_schema_change") { + def tableName = "schema_change_dup_rollup_regression_test" + + try { + sql """ DROP TABLE IF EXISTS ${tableName} """ + + sql """ + CREATE TABLE ${tableName} ( + `user_id` LARGEINT NOT NULL COMMENT "用户id", + `date` DATE NOT NULL COMMENT "数据灌入日期时间", + `city` VARCHAR(20) COMMENT "用户所在城市", + `age` SMALLINT COMMENT "用户年龄", + `sex` TINYINT COMMENT "用户性别", + `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间", + `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `cost` BIGINT DEFAULT "0" COMMENT "用户总消费", + `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", + `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") + DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + PROPERTIES ( "replication_num" = "1" ); + """ + + //add rollup + def result = "null" + def rollupName = "rollup_cost" + sql "ALTER TABLE ${tableName} ADD ROLLUP ${rollupName}(`user_id`,`date`,`city`,`age`,`sex`, cost);" + while (!result.contains("FINISHED")){ + result = sql "SHOW ALTER TABLE ROLLUP WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;" + result = result.toString() + logger.info("result: ${result}") + if(result.contains("CANCELLED")){ + break + } + Thread.sleep(1000) + } + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20) + """ + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20) + """ + result = sql """ + select count(*) from ${tableName} + """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 1) + assertTrue(result[0][0] == 4, "total columns should be 4 rows") + + // add column + sql """ + ALTER table ${tableName} ADD COLUMN new_column INT default "1" + """ + + sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + + sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`last_visit_date`,`last_update_date`, + `last_visit_date_not_null`,`cost`,`max_dwell_time`,`min_dwell_time`) + VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20) + """ + + result = sql """ SELECT * FROM ${tableName} WHERE user_id=3 """ + + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 12) + assertTrue(result[0][11] == 1, "new add column default value should be 1") + + sql """ INSERT INTO ${tableName} VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """ + + assertTrue(result.size() == 2) + assertTrue(result[0].size() == 12) + assertTrue(result[1][11] == 2, "new add column value is set to 2") + + result = sql """ select count(*) from ${tableName} """ + logger.info("result.size:" + result.size() + " result[0].size:" + result[0].size + " " + result[0][0]) + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 1) + assertTrue(result[0][0] == 6, "total count is 6") + + // drop column + sql """ + ALTER TABLE ${tableName} DROP COLUMN sex + """ + result = "null" + while (!result.contains("FINISHED")){ + result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;" + result = result.toString() + logger.info("result: ${result}") + if(result.contains("CANCELLED")) { + log.info("rollup job is cancelled, result: ${result}".toString()) + break + } + Thread.sleep(1000) + } + + result = sql """ select * from ${tableName} where user_id = 3 """ + assertTrue(result.size() == 2) + assertTrue(result[0].size() == 11) + + sql """ INSERT INTO ${tableName} VALUES + (4, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + + result = sql """ select * from ${tableName} where user_id = 4 """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 11) + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + + Thread.sleep(30 * 1000) + // compaction + String[][] tablets = sql """ show tablets from ${tableName}; """ + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + logger.info("run compaction:" + tablet_id) + StringBuilder sb = new StringBuilder(); + sb.append("curl -X POST http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run?tablet_id=") + sb.append(tablet_id) + sb.append("&compact_type=cumulative") + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + //assertEquals(code, 0) + } + + // wait for all compactions done + for (String[] tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run_status?tablet_id=") + sb.append(tablet_id) + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + result = sql """ select count(*) from ${tableName} """ + assertTrue(result.size() == 1) + assertTrue(result[0][0] == 13) + + + result = sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + assertTrue(result.size() == 2) + assertTrue(result[0].size() == 11) + + int rowCount = 0 + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/show?tablet_id=") + sb.append(tablet_id) + String command = sb.toString() + // wait for cleaning stale_rowsets + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + rowCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + logger.info("size:" + rowCount) + assertTrue(rowCount <= 14) + } finally { + //try_sql("DROP TABLE IF EXISTS ${tableName}") + } + +} diff --git a/regression-test/suites/schema_change/test_dup_vals_schema_change.groovy b/regression-test/suites/schema_change/test_dup_vals_schema_change.groovy new file mode 100644 index 000000000000000..ecf0c7f7dbd35e3 --- /dev/null +++ b/regression-test/suites/schema_change/test_dup_vals_schema_change.groovy @@ -0,0 +1,216 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite ("test_dup_vals_schema_change") { + def tableName = "schema_change_dup_vals_regression_test" + + try { + sql """ DROP TABLE IF EXISTS ${tableName} """ + + sql """ + CREATE TABLE ${tableName} ( + `user_id` LARGEINT NOT NULL COMMENT "用户id", + `date` DATE NOT NULL COMMENT "数据灌入日期时间", + `city` VARCHAR(20) COMMENT "用户所在城市", + `age` SMALLINT COMMENT "用户年龄", + `sex` TINYINT COMMENT "用户性别", + `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间", + `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `cost` BIGINT DEFAULT "0" COMMENT "用户总消费", + `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", + `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") + DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + PROPERTIES ( "replication_num" = "1" ); + """ + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20) + """ + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20) + """ + def result1 = sql """ + select count(*) from ${tableName} + """ + assertTrue(result1.size() == 1) + assertTrue(result1[0].size() == 1) + assertTrue(result1[0][0] == 4, "total columns should be 4 rows") + + // add column + sql """ + ALTER table ${tableName} ADD COLUMN new_column INT default "1" + """ + + sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + + sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`last_visit_date`,`last_update_date`, + `last_visit_date_not_null`,`cost`,`max_dwell_time`,`min_dwell_time`) + VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20) + """ + + def result2 = sql """ SELECT * FROM ${tableName} WHERE user_id=3 """ + + assertTrue(result2.size() == 1) + assertTrue(result2[0].size() == 12) + assertTrue(result2[0][11] == 1, "new add column default value should be 1") + + + sql """ INSERT INTO ${tableName} VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + def result3 = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """ + + assertTrue(result3.size() == 2) + assertTrue(result3[0].size() == 12) + assertTrue(result3[1][11] == 2, "new add column value is set to 2") + + def result4 = sql """ select count(*) from ${tableName} """ + logger.info("result4.size:"+result4.size() + " result4[0].size:" + result4[0].size + " " + result4[0][0]) + assertTrue(result4.size() == 1) + assertTrue(result4[0].size() == 1) + assertTrue(result4[0][0] == 6, "total count is 6") + + // drop column + sql """ + ALTER TABLE ${tableName} DROP COLUMN last_visit_date + """ + def result5 = sql """ select * from ${tableName} where user_id = 3 """ + assertTrue(result5.size() == 2) + assertTrue(result5[0].size() == 11) + + sql """ INSERT INTO ${tableName} VALUES + (4, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + + def result6 = sql """ select * from ${tableName} where user_id = 4 """ + assertTrue(result6.size() == 1) + assertTrue(result6[0].size() == 11) + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + + Thread.sleep(30 * 1000) + // compaction + String[][] tablets = sql """ show tablets from ${tableName}; """ + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + logger.info("run compaction:" + tablet_id) + StringBuilder sb = new StringBuilder(); + sb.append("curl -X POST http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run?tablet_id=") + sb.append(tablet_id) + sb.append("&compact_type=cumulative") + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + //assertEquals(code, 0) + } + + // wait for all compactions done + for (String[] tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run_status?tablet_id=") + sb.append(tablet_id) + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + def result7 = sql """ select count(*) from ${tableName} """ + assertTrue(result7.size() == 1) + assertTrue(result7[0][0] == 13) + + def result8 = sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + assertTrue(result8.size() == 2) + assertTrue(result8[0].size() == 11) + + int rowCount = 0 + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/show?tablet_id=") + sb.append(tablet_id) + String command = sb.toString() + // wait for cleaning stale_rowsets + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + rowCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + logger.info("size:" + rowCount) + assertTrue(rowCount < 10) + } finally { + //try_sql("DROP TABLE IF EXISTS ${tableName}") + } +} diff --git a/regression-test/suites/schema_change/test_uniq_keys_schema_change.groovy b/regression-test/suites/schema_change/test_uniq_keys_schema_change.groovy new file mode 100644 index 000000000000000..b8eb3b91cb72e80 --- /dev/null +++ b/regression-test/suites/schema_change/test_uniq_keys_schema_change.groovy @@ -0,0 +1,227 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite ("test_uniq_keys_schema_change") { + def tableName = "schema_change_uniq_keys_regression_test" + + try { + sql """ DROP TABLE IF EXISTS ${tableName} """ + + sql """ + CREATE TABLE ${tableName} ( + `user_id` LARGEINT NOT NULL COMMENT "用户id", + `date` DATE NOT NULL COMMENT "数据灌入日期时间", + `city` VARCHAR(20) COMMENT "用户所在城市", + `age` SMALLINT COMMENT "用户年龄", + `sex` TINYINT COMMENT "用户性别", + `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间", + `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `cost` BIGINT DEFAULT "0" COMMENT "用户总消费", + `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", + `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") + UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + PROPERTIES ( "replication_num" = "1" ); + """ + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20) + """ + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20) + """ + def result = sql """ + select count(*) from ${tableName} + """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 1) + assertTrue(result[0][0] == 2, "total columns should be 2 rows") + + // add column + sql """ + ALTER table ${tableName} ADD COLUMN new_column INT default "1" + """ + + sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + + sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`last_visit_date`,`last_update_date`, + `last_visit_date_not_null`,`cost`,`max_dwell_time`,`min_dwell_time`) + VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20) + """ + + result = sql """ SELECT * FROM ${tableName} WHERE user_id=3 """ + + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 12) + assertTrue(result[0][11] == 1, "new add column default value should be 1") + + + sql """ INSERT INTO ${tableName} VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """ + + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 12) + assertTrue(result[0][11] == 2, "new add column value is set to 2") + + result = sql """ select count(*) from ${tableName} """ + logger.info("result.size:" + result.size() + " result[0].size:" + result[0].size + " " + result[0][0]) + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 1) + assertTrue(result[0][0] == 3, "total count is 3") + + // drop column will throws exception 'Can not drop key column in Unique data model table' + // sql """ + // ALTER TABLE ${tableName} DROP COLUMN sex + // """ + // result = "null" + // while (!result.contains("FINISHED")){ + // result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;" + // result = result.toString() + // logger.info("result: ${result}") + // if(result.contains("CANCELLED")) { + // log.info("rollup job is cancelled, result: ${result}".toString()) + // break + // } + // Thread.sleep(1000) + // } + // result = sql """ select * from ${tableName} where user_id = 3 """ + // assertTrue(result.size() == 1) + // assertTrue(result[0].size() == 11) + + sql """ INSERT INTO ${tableName} VALUES + (4, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + + result = sql """ select * from ${tableName} where user_id = 4 """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 12) + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + + Thread.sleep(30 * 1000) + // compaction + String[][] tablets = sql """ show tablets from ${tableName}; """ + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + logger.info("run compaction:" + tablet_id) + StringBuilder sb = new StringBuilder(); + sb.append("curl -X POST http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run?tablet_id=") + sb.append(tablet_id) + sb.append("&compact_type=cumulative") + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + //assertEquals(code, 0) + } + + // wait for all compactions done + for (String[] tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run_status?tablet_id=") + sb.append(tablet_id) + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + result = sql """ select count(*) from ${tableName} """ + assertTrue(result.size() == 1) + assertTrue(result[0][0] == 5) + + result = sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 12) + + int rowCount = 0 + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/show?tablet_id=") + sb.append(tablet_id) + String command = sb.toString() + // wait for cleaning stale_rowsets + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + rowCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + logger.info("size:" + rowCount) + assertTrue(rowCount <= 10) + } finally { + //try_sql("DROP TABLE IF EXISTS ${tableName}") + } +} diff --git a/regression-test/suites/schema_change/test_uniq_mv_schema_change.groovy b/regression-test/suites/schema_change/test_uniq_mv_schema_change.groovy new file mode 100644 index 000000000000000..4ea38950f4991f0 --- /dev/null +++ b/regression-test/suites/schema_change/test_uniq_mv_schema_change.groovy @@ -0,0 +1,231 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite ("test_uniq_mv_schema_change") { + def tableName = "schema_change_uniq_mv_regression_test" + + try { + sql """ DROP TABLE IF EXISTS ${tableName} """ + + sql """ + CREATE TABLE ${tableName} ( + `user_id` LARGEINT NOT NULL COMMENT "用户id", + `date` DATE NOT NULL COMMENT "数据灌入日期时间", + `city` VARCHAR(20) COMMENT "用户所在城市", + `age` SMALLINT COMMENT "用户年龄", + `sex` TINYINT COMMENT "用户性别", + `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间", + `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `cost` BIGINT DEFAULT "0" COMMENT "用户总消费", + `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", + `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") + UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + PROPERTIES ( "replication_num" = "1" ); + """ + + //add materialized view + def result = "null" + def mvName = "mv1" + sql "create materialized view ${mvName} as select user_id, date, city, age, sex from ${tableName} group by user_id, date, city, age, sex;" + while (!result.contains("FINISHED")){ + result = sql "SHOW ALTER TABLE MATERIALIZED VIEW WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;" + result = result.toString() + logger.info("result: ${result}") + if(result.contains("CANCELLED")){ + break + } + Thread.sleep(1000) + } + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20) + """ + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20) + """ + result = sql """ + select count(*) from ${tableName} + """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 1) + assertTrue(result[0][0] == 2, "total columns should be 2 rows") + + // add column + sql """ + ALTER table ${tableName} ADD COLUMN new_column INT default "1" + """ + + sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + + sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`last_visit_date`,`last_update_date`, + `last_visit_date_not_null`,`cost`,`max_dwell_time`,`min_dwell_time`) + VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20) + """ + + result = sql """ SELECT * FROM ${tableName} WHERE user_id=3 """ + + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 12) + assertTrue(result[0][11] == 1, "new add column default value should be 1") + + + sql """ INSERT INTO ${tableName} VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """ + + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 12) + assertTrue(result[0][11] == 2, "new add column value is set to 2") + + result = sql """ select count(*) from ${tableName} """ + logger.info("result.size:" + result.size() + " result[0].size:" + result[0].size + " " + result[0][0]) + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 1) + assertTrue(result[0][0] == 3, "total count is 3") + + // drop column + sql """ + ALTER TABLE ${tableName} DROP COLUMN cost + """ + + result = sql """ select * from ${tableName} where user_id = 3 """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 11) + + sql """ INSERT INTO ${tableName} VALUES + (4, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2) + """ + + result = sql """ select * from ${tableName} where user_id = 4 """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 11) + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2) + """ + + Thread.sleep(30 * 1000) + // compaction + String[][] tablets = sql """ show tablets from ${tableName}; """ + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + logger.info("run compaction:" + tablet_id) + StringBuilder sb = new StringBuilder(); + sb.append("curl -X POST http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run?tablet_id=") + sb.append(tablet_id) + sb.append("&compact_type=cumulative") + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + //assertEquals(code, 0) + } + + // wait for all compactions done + for (String[] tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run_status?tablet_id=") + sb.append(tablet_id) + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + result = sql """ select count(*) from ${tableName} """ + assertTrue(result.size() == 1) + assertTrue(result[0][0] == 5) + + result = sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 11) + + int rowCount = 0 + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/show?tablet_id=") + sb.append(tablet_id) + String command = sb.toString() + // wait for cleaning stale_rowsets + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + rowCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + logger.info("size:" + rowCount) + assertTrue(rowCount <= 14) + } finally { + //try_sql("DROP TABLE IF EXISTS ${tableName}") + } +} diff --git a/regression-test/suites/schema_change/test_uniq_rollup_schema_change.groovy b/regression-test/suites/schema_change/test_uniq_rollup_schema_change.groovy new file mode 100644 index 000000000000000..7cd897324b85c6b --- /dev/null +++ b/regression-test/suites/schema_change/test_uniq_rollup_schema_change.groovy @@ -0,0 +1,243 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite ("test_uniq_rollup_schema_change") { + def tableName = "schema_change_uniq_rollup_regression_test" + + try { + sql """ DROP TABLE IF EXISTS ${tableName} """ + + sql """ + CREATE TABLE ${tableName} ( + `user_id` LARGEINT NOT NULL COMMENT "用户id", + `date` DATE NOT NULL COMMENT "数据灌入日期时间", + `city` VARCHAR(20) COMMENT "用户所在城市", + `age` SMALLINT COMMENT "用户年龄", + `sex` TINYINT COMMENT "用户性别", + `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间", + `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `cost` BIGINT DEFAULT "0" COMMENT "用户总消费", + `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", + `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") + UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + PROPERTIES ( "replication_num" = "1" ); + """ + + //add rollup + def result = "null" + def rollupName = "rollup_cost" + sql "ALTER TABLE ${tableName} ADD ROLLUP ${rollupName}(`user_id`,`date`,`city`,`age`,`sex`, cost);" + while (!result.contains("FINISHED")){ + result = sql "SHOW ALTER TABLE ROLLUP WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;" + result = result.toString() + logger.info("result: ${result}") + if(result.contains("CANCELLED")){ + break + } + Thread.sleep(1000) + } + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20) + """ + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20) + """ + result = sql """ + select count(*) from ${tableName} + """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 1) + assertTrue(result[0][0] == 2, "total columns should be 2 rows") + + // add column + sql """ + ALTER table ${tableName} ADD COLUMN new_column INT default "1" + """ + + sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + + sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`last_visit_date`,`last_update_date`, + `last_visit_date_not_null`,`cost`,`max_dwell_time`,`min_dwell_time`) + VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20) + """ + + result = sql """ SELECT * FROM ${tableName} WHERE user_id=3 """ + + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 12) + assertTrue(result[0][11] == 1, "new add column default value should be 1") + + + sql """ INSERT INTO ${tableName} VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """ + + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 12) + assertTrue(result[0][11] == 2, "new add column value is set to 2") + + result = sql """ select count(*) from ${tableName} """ + logger.info("result.size:" + result.size() + " result[0].size:" + result[0].size + " " + result[0][0]) + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 1) + assertTrue(result[0][0] == 3, "total count is 3") + + // drop column + sql """ + ALTER TABLE ${tableName} DROP COLUMN cost + """ + + result = "null" + while (!result.contains("FINISHED")){ + result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;" + result = result.toString() + logger.info("result: ${result}") + if(result.contains("CANCELLED")) { + log.info("rollup job is cancelled, result: ${result}".toString()) + break + } + Thread.sleep(1000) + } + + result = sql """ select * from ${tableName} where user_id = 3 """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 11) + + sql """ INSERT INTO ${tableName} VALUES + (4, '2017-10-01', 'Beijing', 10, 1,'2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2) + """ + + result = sql """ select * from ${tableName} where user_id = 4 """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 11) + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1,'2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2) + """ + + Thread.sleep(30 * 1000) + // compaction + String[][] tablets = sql """ show tablets from ${tableName}; """ + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + logger.info("run compaction:" + tablet_id) + StringBuilder sb = new StringBuilder(); + sb.append("curl -X POST http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run?tablet_id=") + sb.append(tablet_id) + sb.append("&compact_type=cumulative") + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + //assertEquals(code, 0) + } + + // wait for all compactions done + for (String[] tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run_status?tablet_id=") + sb.append(tablet_id) + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + result = sql """ select count(*) from ${tableName} """ + assertTrue(result.size() == 1) + assertTrue(result[0][0] == 5) + + result = sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + assertTrue(result.size() == 1) + assertTrue(result[0].size() == 11) + + int rowCount = 0 + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/show?tablet_id=") + sb.append(tablet_id) + String command = sb.toString() + // wait for cleaning stale_rowsets + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + rowCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + logger.info("size:" + rowCount) + assertTrue(rowCount <= 14) + } finally { + //try_sql("DROP TABLE IF EXISTS ${tableName}") + } +} diff --git a/regression-test/suites/schema_change/test_uniq_vals_schema_change.groovy b/regression-test/suites/schema_change/test_uniq_vals_schema_change.groovy new file mode 100644 index 000000000000000..5bd842bc6f0fa83 --- /dev/null +++ b/regression-test/suites/schema_change/test_uniq_vals_schema_change.groovy @@ -0,0 +1,218 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite ("test_uniq_vals_schema_change") { + def tableName = "schema_change_uniq_vals_regression_test" + + try { + sql """ DROP TABLE IF EXISTS ${tableName} """ + + sql """ + CREATE TABLE ${tableName} ( + `user_id` LARGEINT NOT NULL COMMENT "用户id", + `date` DATE NOT NULL COMMENT "数据灌入日期时间", + `city` VARCHAR(20) COMMENT "用户所在城市", + `age` SMALLINT COMMENT "用户年龄", + `sex` TINYINT COMMENT "用户性别", + `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间", + `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `cost` BIGINT DEFAULT "0" COMMENT "用户总消费", + `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", + `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") + UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + PROPERTIES ( "replication_num" = "1" ); + """ + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20) + """ + + sql """ INSERT INTO ${tableName} VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21) + """ + + sql """ INSERT INTO ${tableName} VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20) + """ + + def result1 = sql """ + select count(*) from ${tableName} + """ + assertTrue(result1.size() == 1) + assertTrue(result1[0].size() == 1) + assertTrue(result1[0][0] == 2, "total columns should be 2 rows") + + // add column + sql """ + ALTER table ${tableName} ADD COLUMN new_column INT default "1" + """ + + sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + + sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`last_visit_date`,`last_update_date`, + `last_visit_date_not_null`,`cost`,`max_dwell_time`,`min_dwell_time`) + VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20) + """ + + def result2 = sql """ SELECT * FROM ${tableName} WHERE user_id=3 """ + + assertTrue(result2.size() == 1) + assertTrue(result2[0].size() == 12) + assertTrue(result2[0][11] == 1, "new add column default value should be 1") + + + sql """ INSERT INTO ${tableName} VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + def result3 = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """ + + assertTrue(result3.size() == 1) + assertTrue(result3[0].size() == 12) + assertTrue(result3[0][11] == 2, "new add column value is set to 2") + + def result4 = sql """ select count(*) from ${tableName} """ + logger.info("result4.size:"+result4.size() + " result4[0].size:" + result4[0].size + " " + result4[0][0]) + assertTrue(result4.size() == 1) + assertTrue(result4[0].size() == 1) + assertTrue(result4[0][0] == 3, "total count is 3") + + // drop column + sql """ + ALTER TABLE ${tableName} DROP COLUMN last_visit_date + """ + def result5 = sql """ select * from ${tableName} where user_id = 3 """ + assertTrue(result5.size() == 1) + assertTrue(result5[0].size() == 11) + + sql """ INSERT INTO ${tableName} VALUES + (4, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + + def result6 = sql """ select * from ${tableName} where user_id = 4 """ + assertTrue(result6.size() == 1) + assertTrue(result6[0].size() == 11) + + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + sql """ INSERT INTO ${tableName} VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2) + """ + + Thread.sleep(30 * 1000) + // compaction + String[][] tablets = sql """ show tablets from ${tableName}; """ + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + logger.info("run compaction:" + tablet_id) + StringBuilder sb = new StringBuilder(); + sb.append("curl -X POST http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run?tablet_id=") + sb.append(tablet_id) + sb.append("&compact_type=cumulative") + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + //assertEquals(code, 0) + } + + // wait for all compactions done + for (String[] tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/run_status?tablet_id=") + sb.append(tablet_id) + + String command = sb.toString() + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + def result7 = sql """ select count(*) from ${tableName} """ + assertTrue(result7.size() == 1) + assertTrue(result7[0][0] == 5) + + def result8 = sql """ SELECT * FROM ${tableName} WHERE user_id=2 """ + assertTrue(result8.size() == 1) + assertTrue(result8[0].size() == 11) + + int rowCount = 0 + for (String[] tablet in tablets) { + String tablet_id = tablet[0] + StringBuilder sb = new StringBuilder(); + sb.append("curl -X GET http://") + sb.append(context.config.beHttpAddress) + sb.append("/api/compaction/show?tablet_id=") + sb.append(tablet_id) + String command = sb.toString() + // wait for cleaning stale_rowsets + process = command.execute() + code = process.waitFor() + err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); + out = process.getText() + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + rowCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + logger.info("size:" + rowCount) + assertTrue(rowCount <= 10) + } finally { + //try_sql("DROP TABLE IF EXISTS ${tableName}") + } + +} diff --git a/regression-test/suites/schema_change/test_update_schema_change.sql b/regression-test/suites/schema_change/test_update_schema_change.sql new file mode 100644 index 000000000000000..e2cd8263c22639f --- /dev/null +++ b/regression-test/suites/schema_change/test_update_schema_change.sql @@ -0,0 +1,50 @@ +DROP TABLE IF EXISTS schema_change_update_regression_test; + +CREATE TABLE schema_change_update_regression_test ( + `user_id` LARGEINT NOT NULL COMMENT "用户id", + `date` DATE NOT NULL COMMENT "数据灌入日期时间", + `city` VARCHAR(20) COMMENT "用户所在城市", + `age` SMALLINT COMMENT "用户年龄", + `sex` TINYINT COMMENT "用户性别", + `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间", + `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间", + `cost` BIGINT DEFAULT "0" COMMENT "用户总消费", + `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间", + `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间") + UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`) + PROPERTIES ( "replication_num" = "1" ); + +INSERT INTO schema_change_update_regression_test VALUES + (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20); + +INSERT INTO schema_change_update_regression_test VALUES + (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21); + +SELECT * FROM schema_change_update_regression_test order by user_id ASC, last_visit_date; + +ALTER table schema_change_update_regression_test ADD COLUMN new_column INT default "1"; + +SELECT * FROM schema_change_update_regression_test order by user_id DESC, last_visit_date; + +UPDATE schema_change_update_regression_test set new_column = 2 where user_id = 1; + +SELECT * FROM schema_change_update_regression_test order by user_id ASC, last_visit_date; + +INSERT INTO schema_change_update_regression_test VALUES + (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20, 2); + +INSERT INTO schema_change_update_regression_test VALUES + (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21, 20); + +UPDATE schema_change_update_regression_test set new_column = 20 where new_column = 2; + +SELECT * FROM schema_change_update_regression_test order by user_id DESC, last_visit_date; + +ALTER TABLE schema_change_update_regression_test DROP COLUMN new_column; + +SELECT * FROM schema_change_update_regression_test order by user_id DESC, last_visit_date; + +UPDATE schema_change_update_regression_test set cost = 20 where user_id = 5; + +SELECT * FROM schema_change_update_regression_test order by user_id DESC, last_visit_date; \ No newline at end of file