From 513d3b4baa445ec0f74a5e9a0e0c7d802f0ad126 Mon Sep 17 00:00:00 2001 From: morningman Date: Thu, 12 Sep 2019 16:36:53 +0800 Subject: [PATCH 1/8] Optimize the load performance for large file --- be/src/common/config.h | 4 + be/src/olap/delta_writer.cpp | 66 +++++-- be/src/olap/delta_writer.h | 27 ++- be/src/olap/memtable.cpp | 6 +- be/src/olap/memtable.h | 4 +- be/src/olap/olap_define.h | 1 + be/src/runtime/CMakeLists.txt | 1 + be/src/runtime/tablet_writer_mgr.cpp | 229 +----------------------- be/src/runtime/tablet_writer_mgr.h | 26 +-- be/src/runtime/tablets_channel.cpp | 255 +++++++++++++++++++++++++++ be/src/runtime/tablets_channel.h | 129 ++++++++++++++ 11 files changed, 473 insertions(+), 275 deletions(-) create mode 100644 be/src/runtime/tablets_channel.cpp create mode 100644 be/src/runtime/tablets_channel.h diff --git a/be/src/common/config.h b/be/src/common/config.h index b386d213b3f657..52fe813d3adc5e 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -465,6 +465,10 @@ namespace config { CONF_Int32(storage_flood_stage_usage_percent, "95"); // 95% // The min bytes that should be left of a data dir CONF_Int64(storage_flood_stage_left_capacity_bytes, "1073741824") // 1GB + // the size of queue for saving immutable memtables. + // set this size larger may reduce the time of waiting memtable flush, + // but will increase memory usage of loading. + // CONF_Int32(memtable_queue_size, "1"); } // namespace config } // namespace doris diff --git a/be/src/olap/delta_writer.cpp b/be/src/olap/delta_writer.cpp index 6499981607d681..51ef19291b594e 100644 --- a/be/src/olap/delta_writer.cpp +++ b/be/src/olap/delta_writer.cpp @@ -26,24 +26,32 @@ namespace doris { -OLAPStatus DeltaWriter::open(WriteRequest* req, DeltaWriter** writer) { - *writer = new DeltaWriter(req); +OLAPStatus DeltaWriter::open( + WriteRequest* req, + BlockingQueue>* flush_queue, + DeltaWriter** writer) { + *writer = new DeltaWriter(req, flush_queue); return OLAP_SUCCESS; } -DeltaWriter::DeltaWriter(WriteRequest* req) +DeltaWriter::DeltaWriter( + WriteRequest* req, + BlockingQueue>* flush_queue) : _req(*req), _tablet(nullptr), _cur_rowset(nullptr), _new_rowset(nullptr), _new_tablet(nullptr), - _rowset_writer(nullptr), _mem_table(nullptr), - _schema(nullptr), _tablet_schema(nullptr), - _delta_written_success(false) {} + _rowset_writer(nullptr), _schema(nullptr), _tablet_schema(nullptr), + _delta_written_success(false), _flush_status(OLAP_SUCCESS), + _flush_queue(flush_queue) { + + _mem_table.reset(); +} DeltaWriter::~DeltaWriter() { if (!_delta_written_success) { _garbage_collection(); } - SAFE_DELETE(_mem_table); + _mem_table.reset(); SAFE_DELETE(_schema); if (_rowset_writer != nullptr) { _rowset_writer->data_dir()->remove_pending_ids(ROWSET_ID_PREFIX + _rowset_writer->rowset_id().to_string()); @@ -132,8 +140,8 @@ OLAPStatus DeltaWriter::init() { _tablet_schema = &(_tablet->tablet_schema()); _schema = new Schema(*_tablet_schema); - _mem_table = new MemTable(_schema, _tablet_schema, _req.slots, - _req.tuple_desc, _tablet->keys_type()); + _mem_table = std::make_shared(_tablet->tablet_id(), _schema, _tablet_schema, _req.slots, _req.tuple_desc, _tablet->keys_type()); + _is_init = true; return OLAP_SUCCESS; } @@ -147,26 +155,48 @@ OLAPStatus DeltaWriter::write(Tuple* tuple) { } _mem_table->insert(tuple); + + // if memtable is full, add it to the flush queue, + // and create a new memtable for incoming data if (_mem_table->memory_usage() >= config::write_buffer_size) { - RETURN_NOT_OK(_mem_table->flush(_rowset_writer.get())); + if (_flush_status.load() != OLAP_SUCCESS) { + // last flush already failed, return error + return _flush_status.load(); + } - SAFE_DELETE(_mem_table); - _mem_table = new MemTable(_schema, _tablet_schema, _req.slots, - _req.tuple_desc, _tablet->keys_type()); + if (!_flush_queue->blocking_put(_mem_table)) { + // queue has been shutdown + return OLAP_ERR_WRITER_FLUSH_QUEUE_SHUTDOWN; + } + // create a new memtable for new incoming data + _mem_table.reset(new MemTable(_tablet->tablet_id(), _schema, _tablet_schema, _req.slots, _req.tuple_desc, _tablet->keys_type())); } return OLAP_SUCCESS; } -OLAPStatus DeltaWriter::close(google::protobuf::RepeatedPtrField* tablet_vec) { +OLAPStatus DeltaWriter::flush() { if (!_is_init) { auto st = init(); if (st != OLAP_SUCCESS) { return st; } } - RETURN_NOT_OK(_mem_table->close(_rowset_writer.get())); - OLAPStatus res = OLAP_SUCCESS; + // if last flush already failed, just return error + RETURN_NOT_OK(_flush_status.load()); + // put last memtable to flush queue + _flush_queue->blocking_put(_mem_table); + return OLAP_SUCCESS; +} + +OLAPStatus DeltaWriter::close(google::protobuf::RepeatedPtrField* tablet_vec) { + // return error if flush failed + OLAPStatus res = _flush_status.load(); + if (res != OLAP_SUCCESS) { + LOG(WARNING) << "failed to flush memtable for tablet: " << _tablet->tablet_id(); + return res; + } + // use rowset meta manager to save meta _cur_rowset = _rowset_writer->build(); if (_cur_rowset == nullptr) { @@ -218,6 +248,10 @@ OLAPStatus DeltaWriter::close(google::protobuf::RepeatedPtrField* t #endif _delta_written_success = true; + + LOG(INFO) << "close delta writer for tablet: " << _tablet->tablet_id() + << ", flush cost(ms): " << _flush_cost_ns / 1000 / 1000 + << ", flush times: " << _flush_time; return OLAP_SUCCESS; } diff --git a/be/src/olap/delta_writer.h b/be/src/olap/delta_writer.h index f9d1587f5ee187..95e583f961e531 100644 --- a/be/src/olap/delta_writer.h +++ b/be/src/olap/delta_writer.h @@ -25,6 +25,7 @@ #include "runtime/tuple.h" #include "gen_cpp/internal_service.pb.h" #include "olap/rowset/rowset_writer.h" +#include "util/blocking_queue.hpp" namespace doris { @@ -53,17 +54,31 @@ struct WriteRequest { class DeltaWriter { public: - static OLAPStatus open(WriteRequest* req, DeltaWriter** writer); + static OLAPStatus open( + WriteRequest* req, + BlockingQueue>* flush_queue, + DeltaWriter** writer); OLAPStatus init(); - DeltaWriter(WriteRequest* req); + DeltaWriter(WriteRequest* req, BlockingQueue>* flush_queue); ~DeltaWriter(); OLAPStatus write(Tuple* tuple); + // flush the last memtable to flush queue, must call it before close + OLAPStatus flush(); OLAPStatus close(google::protobuf::RepeatedPtrField* tablet_vec); OLAPStatus cancel(); int64_t partition_id() const { return _req.partition_id; } + void set_flush_status(OLAPStatus st) { _flush_status.store(st); } + OLAPStatus get_flush_status() { return _flush_status.load(); } + RowsetWriter* rowset_writer() { return _rowset_writer.get(); } + + void update_flush_time(int64_t flush_ns) { + _flush_cost_ns += flush_ns; + _flush_time++; + } + private: void _garbage_collection(); @@ -75,10 +90,16 @@ class DeltaWriter { RowsetSharedPtr _new_rowset; TabletSharedPtr _new_tablet; std::unique_ptr _rowset_writer; - MemTable* _mem_table; + std::shared_ptr _mem_table; Schema* _schema; const TabletSchema* _tablet_schema; bool _delta_written_success; + std::atomic _flush_status; + int64_t _flush_cost_ns; + int64_t _flush_time; + + // queue for saving immable mem tables + BlockingQueue>* _flush_queue; }; } // namespace doris diff --git a/be/src/olap/memtable.cpp b/be/src/olap/memtable.cpp index a0054a89deeeda..761689e340613c 100644 --- a/be/src/olap/memtable.cpp +++ b/be/src/olap/memtable.cpp @@ -26,10 +26,11 @@ namespace doris { -MemTable::MemTable(Schema* schema, const TabletSchema* tablet_schema, +MemTable::MemTable(int64_t tablet_id, Schema* schema, const TabletSchema* tablet_schema, const std::vector* slot_descs, TupleDescriptor* tuple_desc, KeysType keys_type) - : _schema(schema), + : _tablet_id(tablet_id), + _schema(schema), _tablet_schema(tablet_schema), _tuple_desc(tuple_desc), _slot_descs(slot_descs), @@ -59,6 +60,7 @@ size_t MemTable::memory_usage() { void MemTable::insert(Tuple* tuple) { ContiguousRow row(_schema, _tuple_buf); + for (size_t i = 0; i < _slot_descs->size(); ++i) { auto cell = row.cell(i); const SlotDescriptor* slot = (*_slot_descs)[i]; diff --git a/be/src/olap/memtable.h b/be/src/olap/memtable.h index 1a262ce3ddb750..7535c957b3744b 100644 --- a/be/src/olap/memtable.h +++ b/be/src/olap/memtable.h @@ -31,15 +31,17 @@ class RowCursor; class MemTable { public: - MemTable(Schema* schema, const TabletSchema* tablet_schema, + MemTable(int64_t tablet_id, Schema* schema, const TabletSchema* tablet_schema, const std::vector* slot_descs, TupleDescriptor* tuple_desc, KeysType keys_type); ~MemTable(); + int64_t tablet_id() { return _tablet_id; } size_t memory_usage(); void insert(Tuple* tuple); OLAPStatus flush(RowsetWriter* rowset_writer); OLAPStatus close(RowsetWriter* rowset_writer); private: + int64_t _tablet_id; Schema* _schema; const TabletSchema* _tablet_schema; TupleDescriptor* _tuple_desc; diff --git a/be/src/olap/olap_define.h b/be/src/olap/olap_define.h index 3735f1e2c8f104..e2e280a6972e8a 100644 --- a/be/src/olap/olap_define.h +++ b/be/src/olap/olap_define.h @@ -268,6 +268,7 @@ enum OLAPStatus { OLAP_ERR_WRITER_DATA_WRITE_ERROR = -1201, OLAP_ERR_WRITER_ROW_BLOCK_ERROR = -1202, OLAP_ERR_WRITER_SEGMENT_NOT_FINALIZED = -1203, + OLAP_ERR_WRITER_FLUSH_QUEUE_SHUTDOWN = -1204, // RowBlock // [-1300, -1400) diff --git a/be/src/runtime/CMakeLists.txt b/be/src/runtime/CMakeLists.txt index e46afeef3e59e1..9b1cdaa52b7814 100644 --- a/be/src/runtime/CMakeLists.txt +++ b/be/src/runtime/CMakeLists.txt @@ -83,6 +83,7 @@ set(RUNTIME_FILES # export_task_mgr.cpp export_sink.cpp tablet_writer_mgr.cpp + tablets_channel.cpp bufferpool/buffer_allocator.cc bufferpool/buffer_pool.cc bufferpool/reservation_tracker.cc diff --git a/be/src/runtime/tablet_writer_mgr.cpp b/be/src/runtime/tablet_writer_mgr.cpp index 0d30dbbab64fe7..1ab49c9c50b7fb 100644 --- a/be/src/runtime/tablet_writer_mgr.cpp +++ b/be/src/runtime/tablet_writer_mgr.cpp @@ -28,228 +28,12 @@ #include "runtime/row_batch.h" #include "runtime/tuple_row.h" #include "service/backend_options.h" -#include "util/bitmap.h" #include "util/stopwatch.hpp" #include "olap/delta_writer.h" #include "olap/lru_cache.h" namespace doris { -// channel that process all data for this load -class TabletsChannel { -public: - TabletsChannel(const TabletsChannelKey& key) : _key(key), _closed_senders(64) { } - ~TabletsChannel(); - - Status open(const PTabletWriterOpenRequest& params); - - Status add_batch(const PTabletWriterAddBatchRequest& batch); - - Status close(int sender_id, bool* finished, - const google::protobuf::RepeatedField& partition_ids, - google::protobuf::RepeatedPtrField* tablet_vec); - - time_t last_updated_time() { - return _last_updated_time; - } - -private: - // open all writer - Status _open_all_writers(const PTabletWriterOpenRequest& params); - -private: - // id of this load channel, just for - TabletsChannelKey _key; - - // make execute sequece - std::mutex _lock; - - // initialized in open function - int64_t _txn_id = -1; - int64_t _index_id = -1; - OlapTableSchemaParam* _schema = nullptr; - TupleDescriptor* _tuple_desc = nullptr; - // row_desc used to construct - RowDescriptor* _row_desc = nullptr; - bool _opened = false; - - // next sequence we expect - int _num_remaining_senders = 0; - std::vector _next_seqs; - Bitmap _closed_senders; - Status _close_status; - - // tablet_id -> TabletChannel - std::unordered_map _tablet_writers; - - std::unordered_set _partition_ids; - - // TODO(zc): to add this tracker to somewhere - MemTracker _mem_tracker; - - //use to erase timeout TabletsChannel in _tablets_channels - time_t _last_updated_time; -}; - -TabletsChannel::~TabletsChannel() { - for (auto& it : _tablet_writers) { - delete it.second; - } - delete _row_desc; - delete _schema; -} - -Status TabletsChannel::open(const PTabletWriterOpenRequest& params) { - std::lock_guard l(_lock); - if (_opened) { - // Normal case, already open by other sender - return Status::OK(); - } - LOG(INFO) << "open tablets channel: " << _key; - _txn_id = params.txn_id(); - _index_id = params.index_id(); - _schema = new OlapTableSchemaParam(); - RETURN_IF_ERROR(_schema->init(params.schema())); - _tuple_desc = _schema->tuple_desc(); - _row_desc = new RowDescriptor(_tuple_desc, false); - - _num_remaining_senders = params.num_senders(); - _next_seqs.resize(_num_remaining_senders, 0); - _closed_senders.Reset(_num_remaining_senders); - - RETURN_IF_ERROR(_open_all_writers(params)); - - _opened = true; - _last_updated_time = time(nullptr); - return Status::OK(); -} - -Status TabletsChannel::add_batch(const PTabletWriterAddBatchRequest& params) { - DCHECK(params.tablet_ids_size() == params.row_batch().num_rows()); - std::lock_guard l(_lock); - DCHECK(_opened); - auto next_seq = _next_seqs[params.sender_id()]; - // check packet - if (params.packet_seq() < next_seq) { - LOG(INFO) << "packet has already recept before, expect_seq=" << next_seq - << ", recept_seq=" << params.packet_seq(); - return Status::OK(); - } else if (params.packet_seq() > next_seq) { - LOG(WARNING) << "lost data packet, expect_seq=" << next_seq - << ", recept_seq=" << params.packet_seq(); - return Status::InternalError("lost data packet"); - } - - RowBatch row_batch(*_row_desc, params.row_batch(), &_mem_tracker); - - // iterator all data - for (int i = 0; i < params.tablet_ids_size(); ++i) { - auto tablet_id = params.tablet_ids(i); - auto it = _tablet_writers.find(tablet_id); - if (it == std::end(_tablet_writers)) { - std::stringstream ss; - ss << "unknown tablet to append data, tablet=" << tablet_id; - return Status::InternalError(ss.str()); - } - auto st = it->second->write(row_batch.get_row(i)->get_tuple(0)); - if (st != OLAP_SUCCESS) { - std::stringstream ss; - ss << "tablet writer write failed, tablet_id=" << it->first - << ", transaction_id=" << _txn_id << ", err=" << st; - LOG(WARNING) << ss.str(); - return Status::InternalError(ss.str()); - } - } - _next_seqs[params.sender_id()]++; - _last_updated_time = time(nullptr); - return Status::OK(); -} - -Status TabletsChannel::close(int sender_id, bool* finished, - const google::protobuf::RepeatedField& partition_ids, - google::protobuf::RepeatedPtrField* tablet_vec) { - std::lock_guard l(_lock); - if (_closed_senders.Get(sender_id)) { - // Double close from one sender, just return OK - *finished = (_num_remaining_senders == 0); - return _close_status; - } - LOG(INFO) << "close tablets channel: " << _key << ", sender id: " << sender_id; - for (auto pid : partition_ids) { - _partition_ids.emplace(pid); - } - _closed_senders.Set(sender_id, true); - _num_remaining_senders--; - *finished = (_num_remaining_senders == 0); - if (*finished) { - // All senders are closed - for (auto& it : _tablet_writers) { - if (_partition_ids.count(it.second->partition_id()) > 0) { - auto st = it.second->close(tablet_vec); - if (st != OLAP_SUCCESS) { - std::stringstream ss; - ss << "close tablet writer failed, tablet_id=" << it.first - << ", transaction_id=" << _txn_id << ", err=" << st; - LOG(WARNING) << ss.str(); - _close_status = Status::InternalError(ss.str()); - return _close_status; - } - } else { - auto st = it.second->cancel(); - if (st != OLAP_SUCCESS) { - LOG(WARNING) << "cancel tablet writer failed, tablet_id=" << it.first - << ", transaction_id=" << _txn_id; - } - } - } - } - return Status::OK(); -} - -Status TabletsChannel::_open_all_writers(const PTabletWriterOpenRequest& params) { - std::vector* index_slots = nullptr; - int32_t schema_hash = 0; - for (auto& index : _schema->indexes()) { - if (index->index_id == _index_id) { - index_slots = &index->slots; - schema_hash = index->schema_hash; - break; - } - } - if (index_slots == nullptr) { - std::stringstream ss; - ss << "unknown index id, key=" << _key; - return Status::InternalError(ss.str()); - } - for (auto& tablet : params.tablets()) { - WriteRequest request; - request.tablet_id = tablet.tablet_id(); - request.schema_hash = schema_hash; - request.write_type = LOAD; - request.txn_id = _txn_id; - request.partition_id = tablet.partition_id(); - request.load_id = params.id(); - request.need_gen_rollup = params.need_gen_rollup(); - request.tuple_desc = _tuple_desc; - request.slots = index_slots; - - DeltaWriter* writer = nullptr; - auto st = DeltaWriter::open(&request, &writer); - if (st != OLAP_SUCCESS) { - std::stringstream ss; - ss << "open delta writer failed, tablet_id=" << tablet.tablet_id() - << ", txn_id=" << _txn_id - << ", partition_id=" << tablet.partition_id() - << ", err=" << st; - LOG(WARNING) << ss.str(); - return Status::InternalError(ss.str()); - } - _tablet_writers.emplace(tablet.tablet_id(), writer); - } - DCHECK(_tablet_writers.size() == params.tablets_size()); - return Status::OK(); -} - TabletWriterMgr::TabletWriterMgr(ExecEnv* exec_env) :_exec_env(exec_env) { _tablets_channels.init(2011); _lastest_success_channel = new_lru_cache(1024); @@ -269,7 +53,7 @@ Status TabletWriterMgr::open(const PTabletWriterOpenRequest& params) { channel = *val; } else { // create a new - channel.reset(new TabletsChannel(key)); + channel.reset(new TabletsChannel(key, params.tablets().size())); _tablets_channels.insert(key, channel); } } @@ -381,15 +165,4 @@ Status TabletWriterMgr::_start_tablets_channel_clean() { return Status::OK(); } -std::string TabletsChannelKey::to_string() const { - std::stringstream ss; - ss << *this; - return ss.str(); -} - -std::ostream& operator<<(std::ostream& os, const TabletsChannelKey& key) { - os << "(id=" << key.id << ",index_id=" << key.index_id << ")"; - return os; -} - } diff --git a/be/src/runtime/tablet_writer_mgr.h b/be/src/runtime/tablet_writer_mgr.h index f5714dade43c04..8a7f1e192e8c5e 100644 --- a/be/src/runtime/tablet_writer_mgr.h +++ b/be/src/runtime/tablet_writer_mgr.h @@ -29,6 +29,7 @@ #include "gen_cpp/Types_types.h" #include "gen_cpp/PaloInternalService_types.h" #include "gen_cpp/internal_service.pb.h" +#include "runtime/tablets_channel.h" #include "util/hash_util.hpp" #include "util/uid_util.h" @@ -37,29 +38,6 @@ namespace doris { class ExecEnv; -class TabletsChannel; - -struct TabletsChannelKey { - UniqueId id; - int64_t index_id; - - TabletsChannelKey(const PUniqueId& pid, int64_t index_id_) - : id(pid), index_id(index_id_) { } - ~TabletsChannelKey() noexcept { } - - bool operator==(const TabletsChannelKey& rhs) const noexcept { - return index_id == rhs.index_id && id == rhs.id; - } - - std::string to_string() const; -}; - -struct TabletsChannelKeyHasher { - std::size_t operator()(const TabletsChannelKey& key) const { - size_t seed = key.id.hash(); - return doris::HashUtil::hash(&key.index_id, sizeof(key.index_id), seed); - } -}; class Cache; @@ -104,6 +82,4 @@ class TabletWriterMgr { Status _start_tablets_channel_clean(); }; -std::ostream& operator<<(std::ostream& os, const TabletsChannelKey&); - } diff --git a/be/src/runtime/tablets_channel.cpp b/be/src/runtime/tablets_channel.cpp new file mode 100644 index 00000000000000..abfd3cda4462e0 --- /dev/null +++ b/be/src/runtime/tablets_channel.cpp @@ -0,0 +1,255 @@ +// 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 "runtime/tablets_channel.h" + +#include "exec/tablet_info.h" +#include "olap/delta_writer.h" +#include "olap/memtable.h" +#include "runtime/row_batch.h" +#include "runtime/tuple_row.h" + +namespace doris { + +TabletsChannel::TabletsChannel(const TabletsChannelKey& key, size_t tablet_num): + _key(key), _closed_senders(64), + _flush_pool(1, 1), + _flush_queue(tablet_num) { +} + +TabletsChannel::~TabletsChannel() { + for (auto& it : _tablet_writers) { + delete it.second; + } + delete _row_desc; + delete _schema; +} + +Status TabletsChannel::open(const PTabletWriterOpenRequest& params) { + std::lock_guard l(_lock); + if (_opened) { + // Normal case, already open by other sender + return Status::OK(); + } + LOG(INFO) << "open tablets channel: " << _key; + _txn_id = params.txn_id(); + _index_id = params.index_id(); + _schema = new OlapTableSchemaParam(); + RETURN_IF_ERROR(_schema->init(params.schema())); + _tuple_desc = _schema->tuple_desc(); + _row_desc = new RowDescriptor(_tuple_desc, false); + + _num_remaining_senders = params.num_senders(); + _next_seqs.resize(_num_remaining_senders, 0); + _closed_senders.Reset(_num_remaining_senders); + + RETURN_IF_ERROR(_open_all_writers(params)); + + _flush_pool.offer(boost::bind(&TabletsChannel::_flush_memtable, this)); + + _opened = true; + _last_updated_time = time(nullptr); + return Status::OK(); +} + +void TabletsChannel::_flush_memtable() { + while(true) { + std::shared_ptr mem; + if (!_flush_queue.blocking_get(&mem)) { + // queue is empty and shutdown, end of thread + return; + } + // find rowset writer for this memtable + int64_t tablet_id = mem->tablet_id(); + auto it = _tablet_writers.find(tablet_id); + if (it == std::end(_tablet_writers)) { + // this should not happen, just discard this memtable. + LOG(WARNING) << "unknown tablet when try flushing memtable, tablet=" << tablet_id; + continue; + } + + // if last flush of this tablet already failed, just skip + if (it->second->get_flush_status() != OLAP_SUCCESS) { + continue; + } + + // flush the memtable + MonotonicStopWatch timer; + timer.start(); + OLAPStatus st = mem->flush(it->second->rowset_writer()); + if (st != OLAP_SUCCESS) { + it->second->set_flush_status(st); + } else { + it->second->update_flush_time(timer.elapsed_time()); + } + mem.reset(); + } +} + +Status TabletsChannel::add_batch(const PTabletWriterAddBatchRequest& params) { + DCHECK(params.tablet_ids_size() == params.row_batch().num_rows()); + std::lock_guard l(_lock); + DCHECK(_opened); + auto next_seq = _next_seqs[params.sender_id()]; + // check packet + if (params.packet_seq() < next_seq) { + LOG(INFO) << "packet has already recept before, expect_seq=" << next_seq + << ", recept_seq=" << params.packet_seq(); + return Status::OK(); + } else if (params.packet_seq() > next_seq) { + LOG(WARNING) << "lost data packet, expect_seq=" << next_seq + << ", recept_seq=" << params.packet_seq(); + return Status::InternalError("lost data packet"); + } + + RowBatch row_batch(*_row_desc, params.row_batch(), &_mem_tracker); + + // iterator all data + for (int i = 0; i < params.tablet_ids_size(); ++i) { + auto tablet_id = params.tablet_ids(i); + auto it = _tablet_writers.find(tablet_id); + if (it == std::end(_tablet_writers)) { + std::stringstream ss; + ss << "unknown tablet to append data, tablet=" << tablet_id; + return Status::InternalError(ss.str()); + } + auto st = it->second->write(row_batch.get_row(i)->get_tuple(0)); + if (st != OLAP_SUCCESS) { + std::stringstream ss; + ss << "tablet writer write failed, tablet_id=" << it->first + << ", transaction_id=" << _txn_id << ", err=" << st; + LOG(WARNING) << ss.str(); + return Status::InternalError(ss.str()); + } + } + _next_seqs[params.sender_id()]++; + _last_updated_time = time(nullptr); + return Status::OK(); +} + +Status TabletsChannel::close(int sender_id, bool* finished, + const google::protobuf::RepeatedField& partition_ids, + google::protobuf::RepeatedPtrField* tablet_vec) { + std::lock_guard l(_lock); + if (_closed_senders.Get(sender_id)) { + // Double close from one sender, just return OK + *finished = (_num_remaining_senders == 0); + return _close_status; + } + LOG(INFO) << "close tablets channel: " << _key << ", sender id: " << sender_id; + for (auto pid : partition_ids) { + _partition_ids.emplace(pid); + } + _closed_senders.Set(sender_id, true); + _num_remaining_senders--; + *finished = (_num_remaining_senders == 0); + if (*finished) { + // All senders are closed + // 1. flush all delta writers + std::vector need_wait_writers; + for (auto& it : _tablet_writers) { + if (_partition_ids.count(it.second->partition_id()) > 0) { + auto st = it.second->flush(); + if (st != OLAP_SUCCESS) { + LOG(WARNING) << "close tablet writer failed, tablet_id=" << it.first + << ", transaction_id=" << _txn_id << ", err=" << st; + // just skip this tablet(writer) and continue to close others + continue; + } + need_wait_writers.push_back(it.second); + } else { + auto st = it.second->cancel(); + if (st != OLAP_SUCCESS) { + LOG(WARNING) << "cancel tablet writer failed, tablet_id=" << it.first + << ", transaction_id=" << _txn_id; + // just skip this tablet(writer) and continue to close others + continue; + } + } + } + + // 2. shutdown the flush queue and wait for all writers to be finished + _flush_queue.shutdown(); + _flush_pool.shutdown(); + _flush_pool.join(); + + // 3. close delta writers and build the tablet vector + for (auto writer : need_wait_writers) { + // close may return failed, but no need to handle it here. + // tablet_vec will only contains success tablet, and then let FE judge it. + writer->close(tablet_vec); + } + } + return Status::OK(); +} + +Status TabletsChannel::_open_all_writers(const PTabletWriterOpenRequest& params) { + std::vector* index_slots = nullptr; + int32_t schema_hash = 0; + for (auto& index : _schema->indexes()) { + if (index->index_id == _index_id) { + index_slots = &index->slots; + schema_hash = index->schema_hash; + break; + } + } + if (index_slots == nullptr) { + std::stringstream ss; + ss << "unknown index id, key=" << _key; + return Status::InternalError(ss.str()); + } + for (auto& tablet : params.tablets()) { + WriteRequest request; + request.tablet_id = tablet.tablet_id(); + request.schema_hash = schema_hash; + request.write_type = LOAD; + request.txn_id = _txn_id; + request.partition_id = tablet.partition_id(); + request.load_id = params.id(); + request.need_gen_rollup = params.need_gen_rollup(); + request.tuple_desc = _tuple_desc; + request.slots = index_slots; + + DeltaWriter* writer = nullptr; + auto st = DeltaWriter::open(&request, &_flush_queue, &writer); + if (st != OLAP_SUCCESS) { + std::stringstream ss; + ss << "open delta writer failed, tablet_id=" << tablet.tablet_id() + << ", txn_id=" << _txn_id + << ", partition_id=" << tablet.partition_id() + << ", err=" << st; + LOG(WARNING) << ss.str(); + return Status::InternalError(ss.str()); + } + _tablet_writers.emplace(tablet.tablet_id(), writer); + } + DCHECK(_tablet_writers.size() == params.tablets_size()); + return Status::OK(); +} + +std::string TabletsChannelKey::to_string() const { + std::stringstream ss; + ss << *this; + return ss.str(); +} + +std::ostream& operator<<(std::ostream& os, const TabletsChannelKey& key) { + os << "(id=" << key.id << ",index_id=" << key.index_id << ")"; + return os; +} + +} diff --git a/be/src/runtime/tablets_channel.h b/be/src/runtime/tablets_channel.h new file mode 100644 index 00000000000000..558cf000b3046d --- /dev/null +++ b/be/src/runtime/tablets_channel.h @@ -0,0 +1,129 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include +#include +#include +#include + +#include "runtime/descriptors.h" +#include "runtime/mem_tracker.h" +#include "util/bitmap.h" +#include "util/thread_pool.hpp" + +#include "gen_cpp/Types_types.h" +#include "gen_cpp/PaloInternalService_types.h" +#include "gen_cpp/internal_service.pb.h" + +namespace doris { + +struct TabletsChannelKey { + UniqueId id; + int64_t index_id; + + TabletsChannelKey(const PUniqueId& pid, int64_t index_id_) + : id(pid), index_id(index_id_) { } + + ~TabletsChannelKey() noexcept { } + + bool operator==(const TabletsChannelKey& rhs) const noexcept { + return index_id == rhs.index_id && id == rhs.id; + } + + std::string to_string() const; +}; + +struct TabletsChannelKeyHasher { + std::size_t operator()(const TabletsChannelKey& key) const { + size_t seed = key.id.hash(); + return doris::HashUtil::hash(&key.index_id, sizeof(key.index_id), seed); + } +}; + +class DeltaWriter; +class MemTable; +class OlapTableSchemaParam; + +// channel that process all data for this load +class TabletsChannel { +public: + TabletsChannel(const TabletsChannelKey& key, size_t tablet_num); + + ~TabletsChannel(); + + Status open(const PTabletWriterOpenRequest& params); + + Status add_batch(const PTabletWriterAddBatchRequest& batch); + + Status close(int sender_id, bool* finished, + const google::protobuf::RepeatedField& partition_ids, + google::protobuf::RepeatedPtrField* tablet_vec); + + time_t last_updated_time() { + return _last_updated_time; + } + +private: + // open all writer + Status _open_all_writers(const PTabletWriterOpenRequest& params); + // the work function of flush thread. + // it will continuously get memtable from flush queue, and flush them to disk + void _flush_memtable(); + +private: + // id of this load channel, just for + TabletsChannelKey _key; + + // make execute sequece + std::mutex _lock; + + // initialized in open function + int64_t _txn_id = -1; + int64_t _index_id = -1; + OlapTableSchemaParam* _schema = nullptr; + TupleDescriptor* _tuple_desc = nullptr; + // row_desc used to construct + RowDescriptor* _row_desc = nullptr; + bool _opened = false; + + // next sequence we expect + int _num_remaining_senders = 0; + std::vector _next_seqs; + Bitmap _closed_senders; + Status _close_status; + + // tablet_id -> TabletChannel + std::unordered_map _tablet_writers; + + std::unordered_set _partition_ids; + + // TODO(zc): to add this tracker to somewhere + MemTracker _mem_tracker; + + //use to erase timeout TabletsChannel in _tablets_channels + time_t _last_updated_time; + + ThreadPool _flush_pool; + // the size of flush queue equals to the number of tablets. + // so that each tablet has at least one rotational memtable. + // and the over all mem usage is at most 2 times of total memtable's size + BlockingQueue> _flush_queue; +}; + +std::ostream& operator<<(std::ostream& os, const TabletsChannelKey& key); + +} // end namespace From 8d9ec6517ab9e339354de0dcf38c7d34d586b9a0 Mon Sep 17 00:00:00 2001 From: morningman Date: Thu, 12 Sep 2019 19:05:48 +0800 Subject: [PATCH 2/8] fix review by kks --- be/src/common/config.h | 4 ---- be/src/olap/delta_writer.cpp | 4 ++-- be/src/olap/delta_writer.h | 8 ++++---- be/src/runtime/tablets_channel.h | 2 +- 4 files changed, 7 insertions(+), 11 deletions(-) diff --git a/be/src/common/config.h b/be/src/common/config.h index 52fe813d3adc5e..b386d213b3f657 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -465,10 +465,6 @@ namespace config { CONF_Int32(storage_flood_stage_usage_percent, "95"); // 95% // The min bytes that should be left of a data dir CONF_Int64(storage_flood_stage_left_capacity_bytes, "1073741824") // 1GB - // the size of queue for saving immutable memtables. - // set this size larger may reduce the time of waiting memtable flush, - // but will increase memory usage of loading. - // CONF_Int32(memtable_queue_size, "1"); } // namespace config } // namespace doris diff --git a/be/src/olap/delta_writer.cpp b/be/src/olap/delta_writer.cpp index 51ef19291b594e..03ddf4059a5d4b 100644 --- a/be/src/olap/delta_writer.cpp +++ b/be/src/olap/delta_writer.cpp @@ -250,8 +250,8 @@ OLAPStatus DeltaWriter::close(google::protobuf::RepeatedPtrField* t _delta_written_success = true; LOG(INFO) << "close delta writer for tablet: " << _tablet->tablet_id() - << ", flush cost(ms): " << _flush_cost_ns / 1000 / 1000 - << ", flush times: " << _flush_time; + << ", flush time(ms): " << _flush_time_ns / 1000 / 1000 + << ", flush count: " << _flush_count; return OLAP_SUCCESS; } diff --git a/be/src/olap/delta_writer.h b/be/src/olap/delta_writer.h index 95e583f961e531..a8266501e41854 100644 --- a/be/src/olap/delta_writer.h +++ b/be/src/olap/delta_writer.h @@ -75,8 +75,8 @@ class DeltaWriter { RowsetWriter* rowset_writer() { return _rowset_writer.get(); } void update_flush_time(int64_t flush_ns) { - _flush_cost_ns += flush_ns; - _flush_time++; + _flush_time_ns += flush_ns; + _flush_count++; } private: @@ -95,8 +95,8 @@ class DeltaWriter { const TabletSchema* _tablet_schema; bool _delta_written_success; std::atomic _flush_status; - int64_t _flush_cost_ns; - int64_t _flush_time; + int64_t _flush_time_ns; + int64_t _flush_count; // queue for saving immable mem tables BlockingQueue>* _flush_queue; diff --git a/be/src/runtime/tablets_channel.h b/be/src/runtime/tablets_channel.h index 558cf000b3046d..dc4961243a77ae 100644 --- a/be/src/runtime/tablets_channel.h +++ b/be/src/runtime/tablets_channel.h @@ -85,7 +85,7 @@ class TabletsChannel { void _flush_memtable(); private: - // id of this load channel, just for + // id of this load channel TabletsChannelKey _key; // make execute sequece From 029316283eebc34b61274b2edd20b876d049b211 Mon Sep 17 00:00:00 2001 From: chenmingyu Date: Tue, 17 Sep 2019 14:23:20 +0800 Subject: [PATCH 3/8] Add memtable flush executor --- be/src/common/config.h | 2 + be/src/olap/delta_writer.cpp | 55 +++--- be/src/olap/delta_writer.h | 35 +++- be/src/olap/memtable.h | 8 + be/src/olap/olap_define.h | 1 - be/src/runtime/CMakeLists.txt | 165 +++++++++--------- be/src/runtime/exec_env.h | 3 + be/src/runtime/exec_env_init.cpp | 3 + be/src/runtime/memtable_flush_executor.cpp | 128 ++++++++++++++ be/src/runtime/memtable_flush_executor.h | 85 +++++++++ be/src/runtime/tablet_writer_mgr.cpp | 3 +- be/src/runtime/tablets_channel.cpp | 51 +----- be/src/runtime/tablets_channel.h | 13 +- be/src/service/doris_main.cpp | 2 + be/test/olap/delta_writer_test.cpp | 19 +- be/test/runtime/CMakeLists.txt | 1 + .../external_scan_context_mgr_test.cpp | 4 +- .../runtime/memtable_flush_executor_test.cpp | 129 ++++++++++++++ be/test/runtime/tablet_writer_mgr_test.cpp | 15 +- be/test/runtime/user_function_cache_test.cpp | 16 +- run-ut.sh | 1 + 21 files changed, 553 insertions(+), 186 deletions(-) create mode 100644 be/src/runtime/memtable_flush_executor.cpp create mode 100644 be/src/runtime/memtable_flush_executor.h create mode 100644 be/test/runtime/memtable_flush_executor_test.cpp diff --git a/be/src/common/config.h b/be/src/common/config.h index b386d213b3f657..b37e788f65ad92 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -465,6 +465,8 @@ namespace config { CONF_Int32(storage_flood_stage_usage_percent, "95"); // 95% // The min bytes that should be left of a data dir CONF_Int64(storage_flood_stage_left_capacity_bytes, "1073741824") // 1GB + // number of thread for flushing memtable per store + CONF_Int32(flush_thread_num_per_store, "2"); } // namespace config } // namespace doris diff --git a/be/src/olap/delta_writer.cpp b/be/src/olap/delta_writer.cpp index 03ddf4059a5d4b..55db72d00c5704 100644 --- a/be/src/olap/delta_writer.cpp +++ b/be/src/olap/delta_writer.cpp @@ -23,25 +23,26 @@ #include "olap/rowset/rowset_factory.h" #include "olap/rowset/rowset_meta_manager.h" #include "olap/rowset/rowset_id_generator.h" +#include "runtime/memtable_flush_executor.h" namespace doris { OLAPStatus DeltaWriter::open( WriteRequest* req, - BlockingQueue>* flush_queue, + MemTableFlushExecutor* flush_executor, DeltaWriter** writer) { - *writer = new DeltaWriter(req, flush_queue); + *writer = new DeltaWriter(req, flush_executor); return OLAP_SUCCESS; } DeltaWriter::DeltaWriter( WriteRequest* req, - BlockingQueue>* flush_queue) + MemTableFlushExecutor* flush_executor) : _req(*req), _tablet(nullptr), _cur_rowset(nullptr), _new_rowset(nullptr), _new_tablet(nullptr), _rowset_writer(nullptr), _schema(nullptr), _tablet_schema(nullptr), _delta_written_success(false), _flush_status(OLAP_SUCCESS), - _flush_queue(flush_queue) { + _flush_executor(flush_executor) { _mem_table.reset(); } @@ -56,6 +57,7 @@ DeltaWriter::~DeltaWriter() { if (_rowset_writer != nullptr) { _rowset_writer->data_dir()->remove_pending_ids(ROWSET_ID_PREFIX + _rowset_writer->rowset_id().to_string()); } + LOG(INFO) << "deconstruct delta writer"; } void DeltaWriter::_garbage_collection() { @@ -142,6 +144,8 @@ OLAPStatus DeltaWriter::init() { _schema = new Schema(*_tablet_schema); _mem_table = std::make_shared(_tablet->tablet_id(), _schema, _tablet_schema, _req.slots, _req.tuple_desc, _tablet->keys_type()); + _flush_queue_idx = _flush_executor->get_queue_idx(_tablet->data_dir()->path_hash()); + _is_init = true; return OLAP_SUCCESS; } @@ -156,24 +160,28 @@ OLAPStatus DeltaWriter::write(Tuple* tuple) { _mem_table->insert(tuple); - // if memtable is full, add it to the flush queue, + // if memtable is full, push it to the flush executor, // and create a new memtable for incoming data if (_mem_table->memory_usage() >= config::write_buffer_size) { - if (_flush_status.load() != OLAP_SUCCESS) { - // last flush already failed, return error - return _flush_status.load(); - } - - if (!_flush_queue->blocking_put(_mem_table)) { - // queue has been shutdown - return OLAP_ERR_WRITER_FLUSH_QUEUE_SHUTDOWN; - } + RETURN_NOT_OK(_flush_memtable_async()); // create a new memtable for new incoming data _mem_table.reset(new MemTable(_tablet->tablet_id(), _schema, _tablet_schema, _req.slots, _req.tuple_desc, _tablet->keys_type())); } return OLAP_SUCCESS; } +OLAPStatus DeltaWriter::_flush_memtable_async() { + // last flush already failed, return error + RETURN_NOT_OK(_flush_status.load()); + + MemTableFlushContext ctx; + ctx.memtable = _mem_table; + ctx.delta_writer = this; + ctx.flush_status = &_flush_status; + _flush_future = _flush_executor->push_memtable(_flush_queue_idx, ctx); + return OLAP_SUCCESS; +} + OLAPStatus DeltaWriter::flush() { if (!_is_init) { auto st = init(); @@ -182,20 +190,17 @@ OLAPStatus DeltaWriter::flush() { } } - // if last flush already failed, just return error - RETURN_NOT_OK(_flush_status.load()); - // put last memtable to flush queue - _flush_queue->blocking_put(_mem_table); + RETURN_NOT_OK(_flush_memtable_async()); return OLAP_SUCCESS; } OLAPStatus DeltaWriter::close(google::protobuf::RepeatedPtrField* tablet_vec) { - // return error if flush failed - OLAPStatus res = _flush_status.load(); - if (res != OLAP_SUCCESS) { - LOG(WARNING) << "failed to flush memtable for tablet: " << _tablet->tablet_id(); - return res; - } + // return error if previous flush failed + RETURN_NOT_OK(_flush_status.load()); + + // wait for the last memtable flushed + // flush() is called before close(), so _flush_future is set at least once. + RETURN_NOT_OK(_flush_future.get()); // use rowset meta manager to save meta _cur_rowset = _rowset_writer->build(); @@ -203,7 +208,7 @@ OLAPStatus DeltaWriter::close(google::protobuf::RepeatedPtrField* t LOG(WARNING) << "fail to build rowset"; return OLAP_ERR_MALLOC_ERROR; } - res = StorageEngine::instance()->txn_manager()->commit_txn(_tablet->data_dir()->get_meta(), + OLAPStatus res = StorageEngine::instance()->txn_manager()->commit_txn(_tablet->data_dir()->get_meta(), _req.partition_id, _req.txn_id,_req.tablet_id, _req.schema_hash, _tablet->tablet_uid(), _req.load_id, _cur_rowset, false); if (res != OLAP_SUCCESS && res != OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST) { diff --git a/be/src/olap/delta_writer.h b/be/src/olap/delta_writer.h index a8266501e41854..8a76ceaf5ede5f 100644 --- a/be/src/olap/delta_writer.h +++ b/be/src/olap/delta_writer.h @@ -31,6 +31,7 @@ namespace doris { class SegmentGroup; class MemTable; +class MemTableFlushExecutor; class Schema; enum WriteType { @@ -56,21 +57,27 @@ class DeltaWriter { public: static OLAPStatus open( WriteRequest* req, - BlockingQueue>* flush_queue, + MemTableFlushExecutor* _flush_executor, DeltaWriter** writer); + OLAPStatus init(); - DeltaWriter(WriteRequest* req, BlockingQueue>* flush_queue); + + DeltaWriter(WriteRequest* req, MemTableFlushExecutor* _flush_executor); + ~DeltaWriter(); + OLAPStatus write(Tuple* tuple); // flush the last memtable to flush queue, must call it before close OLAPStatus flush(); + + + OLAPStatus close(google::protobuf::RepeatedPtrField* tablet_vec); OLAPStatus cancel(); int64_t partition_id() const { return _req.partition_id; } - void set_flush_status(OLAPStatus st) { _flush_status.store(st); } OLAPStatus get_flush_status() { return _flush_status.load(); } RowsetWriter* rowset_writer() { return _rowset_writer.get(); } @@ -80,6 +87,9 @@ class DeltaWriter { } private: + // push a full memtable to flush executor + OLAPStatus _flush_memtable_async(); + void _garbage_collection(); private: @@ -94,12 +104,27 @@ class DeltaWriter { Schema* _schema; const TabletSchema* _tablet_schema; bool _delta_written_success; + + // the flush status of previous memtable. + // the default is OLAP_SUCCESS, and once it changes to some ERROR code, + // it will never change back to OLAP_SUCCESS. + // this status will be checked each time the next memtable is going to be flushed, + // so that if the previous flush is already failed, no need to flush next memtable. std::atomic _flush_status; + // the future of the very last memtable flush execution. + // because the flush of this delta writer's memtables are executed serially, + // if the last memtable is flushed, all previous memtables should already be flushed. + // so we only need to wait and block on the last memtable's flush future. + std::future _flush_future; + // total flush time and flush count of memtables int64_t _flush_time_ns; int64_t _flush_count; - // queue for saving immable mem tables - BlockingQueue>* _flush_queue; + MemTableFlushExecutor* _flush_executor; + // the idx of flush queues vector in MemTableFlushExecutor. + // this idx is got from MemTableFlushExecutor, + // and memtables of this delta writer will be pushed to this certain flush queue only. + int32_t _flush_queue_idx; }; } // namespace doris diff --git a/be/src/olap/memtable.h b/be/src/olap/memtable.h index 7535c957b3744b..76f05b11c0ad17 100644 --- a/be/src/olap/memtable.h +++ b/be/src/olap/memtable.h @@ -18,6 +18,7 @@ #ifndef DORIS_BE_SRC_OLAP_MEMTABLE_H #define DORIS_BE_SRC_OLAP_MEMTABLE_H +#include #include #include "olap/schema.h" @@ -40,6 +41,10 @@ class MemTable { void insert(Tuple* tuple); OLAPStatus flush(RowsetWriter* rowset_writer); OLAPStatus close(RowsetWriter* rowset_writer); + + std::future get_flush_future() { return _flush_promise.get_future(); } + void set_flush_status(OLAPStatus st) { _flush_promise.set_value(st); } + private: int64_t _tablet_id; Schema* _schema; @@ -62,6 +67,9 @@ class MemTable { char* _tuple_buf; size_t _schema_size; Table* _skip_list; + + // the promise it to save result status of flush + std::promise _flush_promise; }; // class MemTable } // namespace doris diff --git a/be/src/olap/olap_define.h b/be/src/olap/olap_define.h index e2e280a6972e8a..3735f1e2c8f104 100644 --- a/be/src/olap/olap_define.h +++ b/be/src/olap/olap_define.h @@ -268,7 +268,6 @@ enum OLAPStatus { OLAP_ERR_WRITER_DATA_WRITE_ERROR = -1201, OLAP_ERR_WRITER_ROW_BLOCK_ERROR = -1202, OLAP_ERR_WRITER_SEGMENT_NOT_FINALIZED = -1203, - OLAP_ERR_WRITER_FLUSH_QUEUE_SHUTDOWN = -1204, // RowBlock // [-1300, -1400) diff --git a/be/src/runtime/CMakeLists.txt b/be/src/runtime/CMakeLists.txt index 9b1cdaa52b7814..d38fd7254b790d 100644 --- a/be/src/runtime/CMakeLists.txt +++ b/be/src/runtime/CMakeLists.txt @@ -24,88 +24,89 @@ set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/runtime") set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/runtime") set(RUNTIME_FILES - broker_mgr.cpp - buffered_block_mgr.cpp - buffered_tuple_stream.cpp - buffered_tuple_stream_ir.cpp - buffer_control_block.cpp - merge_sorter.cpp - client_cache.cpp - data_stream_mgr.cpp - data_stream_sender.cpp - datetime_value.cpp - descriptors.cpp - exec_env.cpp - exec_env_init.cpp - user_function_cache.cpp - mem_pool.cpp - plan_fragment_executor.cpp - primitive_type.cpp - pull_load_task_mgr.cpp - raw_value.cpp - raw_value_ir.cpp - result_sink.cpp - result_writer.cpp - result_buffer_mgr.cpp - row_batch.cpp - runtime_state.cpp - string_value.cpp - thread_resource_mgr.cpp - # timestamp_value.cpp - decimal_value.cpp - decimalv2_value.cpp - large_int_value.cpp - tuple.cpp - tuple_row.cpp - vectorized_row_batch.cpp - dpp_writer.cpp - qsorter.cpp - fragment_mgr.cpp - dpp_sink_internal.cpp - data_spliter.cpp - dpp_sink.cpp - etl_job_mgr.cpp - load_path_mgr.cpp - types.cpp - tmp_file_mgr.cc - disk_io_mgr.cc - disk_io_mgr_reader_context.cc - disk_io_mgr_scan_range.cc - buffered_block_mgr2.cc - test_env.cc - mem_tracker.cpp - spill_sorter.cc - sorted_run_merger.cc - data_stream_recvr.cc - buffered_tuple_stream2.cc - buffered_tuple_stream2_ir.cc - buffered_tuple_stream3.cc - # export_task_mgr.cpp - export_sink.cpp - tablet_writer_mgr.cpp - tablets_channel.cpp - bufferpool/buffer_allocator.cc - bufferpool/buffer_pool.cc - bufferpool/reservation_tracker.cc - bufferpool/reservation_util.cc - bufferpool/suballocator.cc - bufferpool/system_allocator.cc - initial_reservations.cc - snapshot_loader.cpp - query_statistics.cpp - message_body_sink.cpp - stream_load/stream_load_context.cpp - stream_load/stream_load_executor.cpp - routine_load/data_consumer.cpp - routine_load/data_consumer_group.cpp - routine_load/data_consumer_pool.cpp - routine_load/routine_load_task_executor.cpp - small_file_mgr.cpp - result_queue_mgr.cpp - memory_scratch_sink.cpp - external_scan_context_mgr.cpp - memory/system_allocator.cpp - memory/chunk_allocator.cpp + broker_mgr.cpp + buffered_block_mgr.cpp + buffered_tuple_stream.cpp + buffered_tuple_stream_ir.cpp + buffer_control_block.cpp + merge_sorter.cpp + client_cache.cpp + data_stream_mgr.cpp + data_stream_sender.cpp + datetime_value.cpp + descriptors.cpp + exec_env.cpp + exec_env_init.cpp + user_function_cache.cpp + mem_pool.cpp + plan_fragment_executor.cpp + primitive_type.cpp + pull_load_task_mgr.cpp + raw_value.cpp + raw_value_ir.cpp + result_sink.cpp + result_writer.cpp + result_buffer_mgr.cpp + row_batch.cpp + runtime_state.cpp + string_value.cpp + thread_resource_mgr.cpp + # timestamp_value.cpp + decimal_value.cpp + decimalv2_value.cpp + large_int_value.cpp + tuple.cpp + tuple_row.cpp + vectorized_row_batch.cpp + dpp_writer.cpp + qsorter.cpp + fragment_mgr.cpp + dpp_sink_internal.cpp + data_spliter.cpp + dpp_sink.cpp + etl_job_mgr.cpp + load_path_mgr.cpp + types.cpp + tmp_file_mgr.cc + disk_io_mgr.cc + disk_io_mgr_reader_context.cc + disk_io_mgr_scan_range.cc + buffered_block_mgr2.cc + test_env.cc + mem_tracker.cpp + spill_sorter.cc + sorted_run_merger.cc + data_stream_recvr.cc + buffered_tuple_stream2.cc + buffered_tuple_stream2_ir.cc + buffered_tuple_stream3.cc + # export_task_mgr.cpp + export_sink.cpp + tablet_writer_mgr.cpp + tablets_channel.cpp + memtable_flush_executor.cpp + bufferpool/buffer_allocator.cc + bufferpool/buffer_pool.cc + bufferpool/reservation_tracker.cc + bufferpool/reservation_util.cc + bufferpool/suballocator.cc + bufferpool/system_allocator.cc + initial_reservations.cc + snapshot_loader.cpp + query_statistics.cpp + message_body_sink.cpp + stream_load/stream_load_context.cpp + stream_load/stream_load_executor.cpp + routine_load/data_consumer.cpp + routine_load/data_consumer_group.cpp + routine_load/data_consumer_pool.cpp + routine_load/routine_load_task_executor.cpp + small_file_mgr.cpp + result_queue_mgr.cpp + memory_scratch_sink.cpp + external_scan_context_mgr.cpp + memory/system_allocator.cpp + memory/chunk_allocator.cpp ) if (WITH_MYSQL) diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index b14be057904f5d..3cfd3a4bb87bb6 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -37,6 +37,7 @@ class FragmentMgr; class LoadPathMgr; class LoadStreamMgr; class MemTracker; +class MemTableFlushExecutor; class MetricRegistry; class StorageEngine; class PoolMemTrackerRegistry; @@ -126,6 +127,7 @@ class ExecEnv { StreamLoadExecutor* stream_load_executor() { return _stream_load_executor; } RoutineLoadTaskExecutor* routine_load_task_executor() { return _routine_load_task_executor; } + MemTableFlushExecutor* memtable_flush_executor() { return _memtable_flush_executor; } private: Status _init(const std::vector& store_paths); @@ -176,6 +178,7 @@ class ExecEnv { StreamLoadExecutor* _stream_load_executor = nullptr; RoutineLoadTaskExecutor* _routine_load_task_executor = nullptr; SmallFileMgr* _small_file_mgr = nullptr; + MemTableFlushExecutor* _memtable_flush_executor = nullptr; }; } diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 87184a04b68fb4..63789093a80c4e 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -30,6 +30,7 @@ #include "runtime/result_buffer_mgr.h" #include "runtime/result_queue_mgr.h" #include "runtime/mem_tracker.h" +#include "runtime/memtable_flush_executor.h" #include "runtime/thread_resource_mgr.h" #include "runtime/fragment_mgr.h" #include "runtime/tablet_writer_mgr.h" @@ -105,6 +106,7 @@ Status ExecEnv::_init(const std::vector& store_paths) { _stream_load_executor = new StreamLoadExecutor(this); _routine_load_task_executor = new RoutineLoadTaskExecutor(this); _small_file_mgr = new SmallFileMgr(this, config::small_file_dir); + _memtable_flush_executor = new MemTableFlushExecutor(this); _backend_client_cache->init_metrics(DorisMetrics::metrics(), "backend"); _frontend_client_cache->init_metrics(DorisMetrics::metrics(), "frontend"); @@ -236,6 +238,7 @@ void ExecEnv::_destory() { delete _stream_load_executor; delete _routine_load_task_executor; delete _external_scan_context_mgr; + delete _memtable_flush_executor; _metrics = nullptr; } diff --git a/be/src/runtime/memtable_flush_executor.cpp b/be/src/runtime/memtable_flush_executor.cpp new file mode 100644 index 00000000000000..256dc39c76fe11 --- /dev/null +++ b/be/src/runtime/memtable_flush_executor.cpp @@ -0,0 +1,128 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "runtime/memtable_flush_executor.h" + +#include "olap/memtable.h" +#include "olap/delta_writer.h" +#include "runtime/exec_env.h" + +namespace doris { + +MemTableFlushExecutor::MemTableFlushExecutor(ExecEnv* exec_env): + _exec_env(exec_env) { +} + +void MemTableFlushExecutor::init() { +#ifndef BE_TEST + int32_t data_dir_num = _exec_env->store_paths().size(); + _thread_num_per_store = std::max(1, config::flush_thread_num_per_store); + _num_threads = data_dir_num * _thread_num_per_store; +#else + int32_t data_dir_num = 1; + _thread_num_per_store = std::max(1, config::flush_thread_num_per_store); + _num_threads = data_dir_num * _thread_num_per_store; +#endif + + // create flush queues + for (int i = 0; i < _num_threads; ++i) { + BlockingQueue* queue = new BlockingQueue(10); + _flush_queues.push_back(queue); + } + // create thread pool + _flush_pool = new ThreadPool(_num_threads, 1); + for (int32_t i = 0; i < _num_threads; ++i) { + _flush_pool->offer(boost::bind(&MemTableFlushExecutor::_flush_memtable, this, i)); + } + + // _path_map saves the path hash to current idx of flush queue. + // eg. + // there are 4 data stores, each store has 2 work thread. + // so there are 8(= 4 * 2) queues in _flush_queues. + // and the path hash of the 4 paths are mapped to idx 0, 2, 4, 6. + int32_t group = 0; + for (auto store : _exec_env->storage_engine()->get_stores()) { + _path_map[store->path_hash()] = group; + group += _thread_num_per_store; + } +} + +MemTableFlushExecutor::~MemTableFlushExecutor() { + // shutdown queues + for (auto queue : _flush_queues) { + queue->shutdown(); + } + + // shutdown thread pool + _flush_pool->shutdown(); + _flush_pool->join(); + + // delete queue + for (auto queue : _flush_queues) { + delete queue; + } + _flush_queues.clear(); + + delete _flush_pool; +} + +int32_t MemTableFlushExecutor::get_queue_idx(size_t path_hash) { + std::lock_guard l(_lock); + int32_t cur_idx = _path_map[path_hash]; + int32_t group = cur_idx / _thread_num_per_store; + int32_t next_idx = group * _thread_num_per_store + ((cur_idx + 1) % _thread_num_per_store); + DCHECK(next_idx < _num_threads); + _path_map[path_hash] = next_idx; + return cur_idx; +} + +std::future MemTableFlushExecutor::push_memtable(int32_t queue_idx, const MemTableFlushContext& ctx) { + _flush_queues[queue_idx]->blocking_put(ctx); + return ctx.memtable->get_flush_future(); +} + +void MemTableFlushExecutor::_flush_memtable(int32_t queue_idx) { + while(true) { + MemTableFlushContext ctx; + if (!_flush_queues[queue_idx]->blocking_get(&ctx)) { + // queue is empty and shutdown, end of thread + return; + } + + DeltaWriter* delta_writer = ctx.delta_writer; + // if last flush of this tablet already failed, just skip + if (delta_writer->get_flush_status() != OLAP_SUCCESS) { + continue; + } + + // flush the memtable + MonotonicStopWatch timer; + timer.start(); + OLAPStatus st = ctx.memtable->flush(delta_writer->rowset_writer()); + if (st == OLAP_SUCCESS) { + delta_writer->update_flush_time(timer.elapsed_time()); + } else { + // if failed, update the flush status, this staus will be observed + // by delta writer. + ctx.flush_status->store(st); + } + // set the promise's value + ctx.memtable->set_flush_status(st); + } +} + +} // end of namespace diff --git a/be/src/runtime/memtable_flush_executor.h b/be/src/runtime/memtable_flush_executor.h new file mode 100644 index 00000000000000..10dca477e7c294 --- /dev/null +++ b/be/src/runtime/memtable_flush_executor.h @@ -0,0 +1,85 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +#include "olap/olap_define.h" +#include "util/blocking_queue.hpp" +#include "util/spinlock.h" +#include "util/thread_pool.hpp" + +namespace doris { + +class ExecEnv; +class DeltaWriter; +class MemTable; + +// The context for a memtable to be flushed. +// It does not own any objects in it. +struct MemTableFlushContext { + std::shared_ptr memtable; + DeltaWriter* delta_writer; + std::atomic* flush_status; +}; + +// MemTableFlushExecutor is for flushing memtables to disk. +// Each data directory has a specified number of worker threads and a corresponding number of flush queues. +// Each worker thread only takes memtable from the corresponding flush queue and writes it to disk. +class MemTableFlushExecutor { +public: + MemTableFlushExecutor(ExecEnv* exec_env); + // init should be called after storage engine is opened, + // because it needs path hash of each data dir. + void init(); + + ~MemTableFlushExecutor(); + + // given the path hash, return the next idx of flush queue. + // eg. + // path A is mapped to idx 0 and 1, so each time get_queue_idx(A) is called, + // 0 and 1 will returned alternately. + int32_t get_queue_idx(size_t path_hash); + + // push the memtable to specified flush queue, and return a future + std::future push_memtable(int32_t queue_idx, const MemTableFlushContext& ctx); + +private: + void _flush_memtable(int32_t queue_idx); + +private: + ExecEnv* _exec_env; + int32_t _thread_num_per_store; + int32_t _num_threads; + ThreadPool* _flush_pool; + // the size of this vector should equals to _num_threads + std::vector*> _flush_queues; + + // lock to protect path_map + SpinLock _lock; + // path hash -> queue idx of _flush_queues; + std::unordered_map _path_map; +}; + +} // end namespace diff --git a/be/src/runtime/tablet_writer_mgr.cpp b/be/src/runtime/tablet_writer_mgr.cpp index 1ab49c9c50b7fb..498b2d1f25e9f6 100644 --- a/be/src/runtime/tablet_writer_mgr.cpp +++ b/be/src/runtime/tablet_writer_mgr.cpp @@ -24,6 +24,7 @@ #include "common/object_pool.h" #include "exec/tablet_info.h" #include "runtime/descriptors.h" +#include "runtime/exec_env.h" #include "runtime/mem_tracker.h" #include "runtime/row_batch.h" #include "runtime/tuple_row.h" @@ -53,7 +54,7 @@ Status TabletWriterMgr::open(const PTabletWriterOpenRequest& params) { channel = *val; } else { // create a new - channel.reset(new TabletsChannel(key, params.tablets().size())); + channel.reset(new TabletsChannel(key, _exec_env->memtable_flush_executor())); _tablets_channels.insert(key, channel); } } diff --git a/be/src/runtime/tablets_channel.cpp b/be/src/runtime/tablets_channel.cpp index abfd3cda4462e0..3333b337208f4f 100644 --- a/be/src/runtime/tablets_channel.cpp +++ b/be/src/runtime/tablets_channel.cpp @@ -25,10 +25,8 @@ namespace doris { -TabletsChannel::TabletsChannel(const TabletsChannelKey& key, size_t tablet_num): - _key(key), _closed_senders(64), - _flush_pool(1, 1), - _flush_queue(tablet_num) { +TabletsChannel::TabletsChannel(const TabletsChannelKey& key, MemTableFlushExecutor* flush_executor): + _key(key), _flush_executor(flush_executor), _closed_senders(64) { } TabletsChannel::~TabletsChannel() { @@ -59,47 +57,11 @@ Status TabletsChannel::open(const PTabletWriterOpenRequest& params) { RETURN_IF_ERROR(_open_all_writers(params)); - _flush_pool.offer(boost::bind(&TabletsChannel::_flush_memtable, this)); - _opened = true; _last_updated_time = time(nullptr); return Status::OK(); } -void TabletsChannel::_flush_memtable() { - while(true) { - std::shared_ptr mem; - if (!_flush_queue.blocking_get(&mem)) { - // queue is empty and shutdown, end of thread - return; - } - // find rowset writer for this memtable - int64_t tablet_id = mem->tablet_id(); - auto it = _tablet_writers.find(tablet_id); - if (it == std::end(_tablet_writers)) { - // this should not happen, just discard this memtable. - LOG(WARNING) << "unknown tablet when try flushing memtable, tablet=" << tablet_id; - continue; - } - - // if last flush of this tablet already failed, just skip - if (it->second->get_flush_status() != OLAP_SUCCESS) { - continue; - } - - // flush the memtable - MonotonicStopWatch timer; - timer.start(); - OLAPStatus st = mem->flush(it->second->rowset_writer()); - if (st != OLAP_SUCCESS) { - it->second->set_flush_status(st); - } else { - it->second->update_flush_time(timer.elapsed_time()); - } - mem.reset(); - } -} - Status TabletsChannel::add_batch(const PTabletWriterAddBatchRequest& params) { DCHECK(params.tablet_ids_size() == params.row_batch().num_rows()); std::lock_guard l(_lock); @@ -182,12 +144,7 @@ Status TabletsChannel::close(int sender_id, bool* finished, } } - // 2. shutdown the flush queue and wait for all writers to be finished - _flush_queue.shutdown(); - _flush_pool.shutdown(); - _flush_pool.join(); - - // 3. close delta writers and build the tablet vector + // 2. wait and close delta writers and build the tablet vector for (auto writer : need_wait_writers) { // close may return failed, but no need to handle it here. // tablet_vec will only contains success tablet, and then let FE judge it. @@ -225,7 +182,7 @@ Status TabletsChannel::_open_all_writers(const PTabletWriterOpenRequest& params) request.slots = index_slots; DeltaWriter* writer = nullptr; - auto st = DeltaWriter::open(&request, &_flush_queue, &writer); + auto st = DeltaWriter::open(&request, _flush_executor, &writer); if (st != OLAP_SUCCESS) { std::stringstream ss; ss << "open delta writer failed, tablet_id=" << tablet.tablet_id() diff --git a/be/src/runtime/tablets_channel.h b/be/src/runtime/tablets_channel.h index dc4961243a77ae..bd9b90b7550d3b 100644 --- a/be/src/runtime/tablets_channel.h +++ b/be/src/runtime/tablets_channel.h @@ -56,12 +56,13 @@ struct TabletsChannelKeyHasher { class DeltaWriter; class MemTable; +class MemTableFlushExecutor; class OlapTableSchemaParam; // channel that process all data for this load class TabletsChannel { public: - TabletsChannel(const TabletsChannelKey& key, size_t tablet_num); + TabletsChannel(const TabletsChannelKey& key, MemTableFlushExecutor* flush_executor); ~TabletsChannel(); @@ -80,13 +81,11 @@ class TabletsChannel { private: // open all writer Status _open_all_writers(const PTabletWriterOpenRequest& params); - // the work function of flush thread. - // it will continuously get memtable from flush queue, and flush them to disk - void _flush_memtable(); private: // id of this load channel TabletsChannelKey _key; + MemTableFlushExecutor* _flush_executor; // make execute sequece std::mutex _lock; @@ -116,12 +115,6 @@ class TabletsChannel { //use to erase timeout TabletsChannel in _tablets_channels time_t _last_updated_time; - - ThreadPool _flush_pool; - // the size of flush queue equals to the number of tablets. - // so that each tablet has at least one rotational memtable. - // and the over all mem usage is at most 2 times of total memtable's size - BlockingQueue> _flush_queue; }; std::ostream& operator<<(std::ostream& os, const TabletsChannelKey& key); diff --git a/be/src/service/doris_main.cpp b/be/src/service/doris_main.cpp index a916039e2fa576..d052a582ac745c 100644 --- a/be/src/service/doris_main.cpp +++ b/be/src/service/doris_main.cpp @@ -39,6 +39,7 @@ #include "common/status.h" #include "codegen/llvm_codegen.h" #include "runtime/exec_env.h" +#include "runtime/memtable_flush_executor.h" #include "util/logging.h" #include "util/network_util.h" #include "util/thrift_util.h" @@ -166,6 +167,7 @@ int main(int argc, char** argv) { auto exec_env = doris::ExecEnv::GetInstance(); doris::ExecEnv::init(exec_env, paths); exec_env->set_storage_engine(engine); + exec_env->memtable_flush_executor()->init(); doris::FrontendHelper::setup(exec_env); doris::ThriftServer* be_server = nullptr; diff --git a/be/test/olap/delta_writer_test.cpp b/be/test/olap/delta_writer_test.cpp index ab4e0aba36e0b3..4b10a059afa434 100644 --- a/be/test/olap/delta_writer_test.cpp +++ b/be/test/olap/delta_writer_test.cpp @@ -30,6 +30,8 @@ #include "olap/utils.h" #include "runtime/tuple.h" #include "runtime/descriptor_helper.h" +#include "runtime/exec_env.h" +#include "runtime/memtable_flush_executor.h" #include "util/logging.h" #include "olap/options.h" #include "olap/tablet_meta_manager.h" @@ -43,6 +45,7 @@ static const uint32_t MAX_RETRY_TIMES = 10; static const uint32_t MAX_PATH_LEN = 1024; StorageEngine* k_engine = nullptr; +MemTableFlushExecutor* k_flush_executor = nullptr; void set_up() { char buffer[MAX_PATH_LEN]; @@ -56,11 +59,19 @@ void set_up() { doris::EngineOptions options; options.store_paths = paths; doris::StorageEngine::open(options, &k_engine); + + ExecEnv* exec_env = doris::ExecEnv::GetInstance(); + exec_env->set_storage_engine(k_engine); + + k_flush_executor = new MemTableFlushExecutor(exec_env); + k_flush_executor->init(); } void tear_down() { delete k_engine; k_engine = nullptr; + delete k_flush_executor; + k_flush_executor = nullptr; system("rm -rf ./data_test"); remove_all_dir(std::string(getenv("DORIS_HOME")) + UNUSED_PREFIX); } @@ -299,8 +310,10 @@ TEST_F(TestDeltaWriter, open) { WriteRequest write_req = {10003, 270068375, WriteType::LOAD, 20001, 30001, load_id, false, tuple_desc}; DeltaWriter* delta_writer = nullptr; - DeltaWriter::open(&write_req, &delta_writer); + DeltaWriter::open(&write_req, k_flush_executor, &delta_writer); ASSERT_NE(delta_writer, nullptr); + res = delta_writer->flush(); + ASSERT_EQ(OLAP_SUCCESS, res); res = delta_writer->close(nullptr); ASSERT_EQ(OLAP_SUCCESS, res); SAFE_DELETE(delta_writer); @@ -332,7 +345,7 @@ TEST_F(TestDeltaWriter, write) { 20002, 30002, load_id, false, tuple_desc, &(tuple_desc->slots())}; DeltaWriter* delta_writer = nullptr; - DeltaWriter::open(&write_req, &delta_writer); + DeltaWriter::open(&write_req, k_flush_executor, &delta_writer); ASSERT_NE(delta_writer, nullptr); Arena arena; @@ -391,6 +404,8 @@ TEST_F(TestDeltaWriter, write) { ASSERT_EQ(OLAP_SUCCESS, res); } + res = delta_writer->flush(); + ASSERT_EQ(OLAP_SUCCESS, res); res = delta_writer->close(nullptr); ASSERT_EQ(OLAP_SUCCESS, res); diff --git a/be/test/runtime/CMakeLists.txt b/be/test/runtime/CMakeLists.txt index ccfe08bc4b6af1..0ca58bd08af863 100644 --- a/be/test/runtime/CMakeLists.txt +++ b/be/test/runtime/CMakeLists.txt @@ -59,6 +59,7 @@ ADD_BE_TEST(user_function_cache_test) ADD_BE_TEST(kafka_consumer_pipe_test) ADD_BE_TEST(routine_load_task_executor_test) ADD_BE_TEST(small_file_mgr_test) +ADD_BE_TEST(memtable_flush_executor_test) ADD_BE_TEST(result_queue_mgr_test) ADD_BE_TEST(memory_scratch_sink_test) diff --git a/be/test/runtime/external_scan_context_mgr_test.cpp b/be/test/runtime/external_scan_context_mgr_test.cpp index 8ab420224581f9..8cb542f4cc670a 100644 --- a/be/test/runtime/external_scan_context_mgr_test.cpp +++ b/be/test/runtime/external_scan_context_mgr_test.cpp @@ -106,8 +106,10 @@ int main(int argc, char** argv) { fprintf(stderr, "error read config file. \n"); return -1; } + + doris::config::scan_context_gc_interval_min = 1; // doris::init_glog("be-test"); ::testing::InitGoogleTest(&argc, argv); doris::CpuInfo::init(); return RUN_ALL_TESTS(); -} \ No newline at end of file +} diff --git a/be/test/runtime/memtable_flush_executor_test.cpp b/be/test/runtime/memtable_flush_executor_test.cpp new file mode 100644 index 00000000000000..79ba2409ba3571 --- /dev/null +++ b/be/test/runtime/memtable_flush_executor_test.cpp @@ -0,0 +1,129 @@ +// 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 "runtime/memtable_flush_executor.h" + +#include +#include +#include + +#include "gen_cpp/Descriptors_types.h" +#include "gen_cpp/PaloInternalService_types.h" +#include "gen_cpp/Types_types.h" +#include "olap/delta_writer.h" +#include "olap/field.h" +#include "olap/memtable.h" +#include "olap/schema.h" +#include "olap/storage_engine.h" +#include "olap/tablet.h" +#include "olap/utils.h" +#include "runtime/tuple.h" +#include "runtime/descriptor_helper.h" +#include "runtime/exec_env.h" +#include "runtime/memtable_flush_executor.h" +#include "util/logging.h" +#include "olap/options.h" +#include "olap/tablet_meta_manager.h" + +namespace doris { + +StorageEngine* k_engine = nullptr; +MemTableFlushExecutor* k_flush_executor = nullptr; + +void set_up() { + char buffer[1024]; + getcwd(buffer, 1024); + config::storage_root_path = std::string(buffer) + "/data_test"; + remove_all_dir(config::storage_root_path); + create_dir(config::storage_root_path); + std::vector paths; + paths.emplace_back(config::storage_root_path, -1); + + doris::EngineOptions options; + options.store_paths = paths; + doris::StorageEngine::open(options, &k_engine); + + ExecEnv* exec_env = doris::ExecEnv::GetInstance(); + exec_env->set_storage_engine(k_engine); + + k_flush_executor = new MemTableFlushExecutor(exec_env); + k_flush_executor->init(); +} + +void tear_down() { + delete k_engine; + k_engine = nullptr; + delete k_flush_executor; + k_flush_executor = nullptr; + system("rm -rf ./data_test"); + remove_all_dir(std::string(getenv("DORIS_HOME")) + UNUSED_PREFIX); +} + +Schema create_schema() { + std::vector col_schemas; + col_schemas.emplace_back(OLAP_FIELD_AGGREGATION_NONE, OLAP_FIELD_TYPE_SMALLINT, true); + col_schemas.emplace_back(OLAP_FIELD_AGGREGATION_NONE, OLAP_FIELD_TYPE_INT, true); + col_schemas.emplace_back(OLAP_FIELD_AGGREGATION_SUM, OLAP_FIELD_TYPE_BIGINT, true); + Schema schema(col_schemas, 2); + return schema; +} + +class TestMemTableFlushExecutor : public ::testing::Test { +public: + TestMemTableFlushExecutor() { } + ~TestMemTableFlushExecutor() { } + + void SetUp() { + std::cout << "setup" << std::endl; + } + + void TearDown(){ + std::cout << "tear down" << std::endl; + } +}; + +TEST_F(TestMemTableFlushExecutor, get_queue_idx) { + for (auto store : k_engine->get_stores()) { + int32_t idx = k_flush_executor->get_queue_idx(store->path_hash()); + ASSERT_EQ(0, idx); + idx = k_flush_executor->get_queue_idx(store->path_hash()); + ASSERT_EQ(1, idx); + idx = k_flush_executor->get_queue_idx(store->path_hash()); + ASSERT_EQ(0, idx); + idx = k_flush_executor->get_queue_idx(store->path_hash()); + ASSERT_EQ(1, idx); + } +} + +} // namespace doris + +int main(int argc, char** argv) { + std::string conffile = std::string(getenv("DORIS_HOME")) + "/conf/be.conf"; + if (!doris::config::init(conffile.c_str(), false)) { + fprintf(stderr, "error read config file. \n"); + return -1; + } + doris::init_glog("be-test"); + int ret = doris::OLAP_SUCCESS; + testing::InitGoogleTest(&argc, argv); + doris::CpuInfo::init(); + doris::set_up(); + ret = RUN_ALL_TESTS(); + doris::tear_down(); + google::protobuf::ShutdownProtobufLibrary(); + return ret; +} diff --git a/be/test/runtime/tablet_writer_mgr_test.cpp b/be/test/runtime/tablet_writer_mgr_test.cpp index 69fce8f91912c4..1fc8240540846f 100644 --- a/be/test/runtime/tablet_writer_mgr_test.cpp +++ b/be/test/runtime/tablet_writer_mgr_test.cpp @@ -27,6 +27,7 @@ #include "runtime/descriptors.h" #include "runtime/primitive_type.h" #include "runtime/mem_tracker.h" +// #include "runtime/memtable_flush_executor.h" #include "runtime/row_batch.h" #include "runtime/tuple_row.h" #include "runtime/descriptor_helper.h" @@ -42,7 +43,7 @@ OLAPStatus close_status; int64_t wait_lock_time_ns; // mock -DeltaWriter::DeltaWriter(WriteRequest* req) : _req(*req) { +DeltaWriter::DeltaWriter(WriteRequest* req, MemTableFlushExecutor* flush_executor) : _req(*req) { } DeltaWriter::~DeltaWriter() { @@ -52,11 +53,11 @@ OLAPStatus DeltaWriter::init() { return OLAP_SUCCESS; } -OLAPStatus DeltaWriter::open(WriteRequest* req, DeltaWriter** writer) { +OLAPStatus DeltaWriter::open(WriteRequest* req, MemTableFlushExecutor* flush_executor, DeltaWriter** writer) { if (open_status != OLAP_SUCCESS) { return open_status; } - *writer = new DeltaWriter(req); + *writer = new DeltaWriter(req, flush_executor); return open_status; } @@ -69,6 +70,10 @@ OLAPStatus DeltaWriter::write(Tuple* tuple) { return add_status; } +OLAPStatus DeltaWriter::flush() { + return OLAP_SUCCESS; +} + OLAPStatus DeltaWriter::close(google::protobuf::RepeatedPtrField* tablet_vec) { return close_status; } @@ -479,7 +484,9 @@ TEST_F(TabletWriterMgrTest, close_failed) { google::protobuf::RepeatedPtrField tablet_vec; auto st = mgr.add_batch(request, &tablet_vec, &wait_lock_time_ns); request.release_id(); - ASSERT_FALSE(st.ok()); + // even if delta close failed, the return status is still ok, but tablet_vec is empty + ASSERT_TRUE(st.ok()); + ASSERT_TRUE(tablet_vec.empty()); } } diff --git a/be/test/runtime/user_function_cache_test.cpp b/be/test/runtime/user_function_cache_test.cpp index b125938f3267a7..a4dc1fc317c19b 100644 --- a/be/test/runtime/user_function_cache_test.cpp +++ b/be/test/runtime/user_function_cache_test.cpp @@ -86,7 +86,7 @@ class UserFunctionCacheTest : public testing::Test { UserFunctionCacheTest() { } virtual ~UserFunctionCacheTest() { } static void SetUpTestCase() { - s_server = new EvHttpServer(29999); + s_server = new EvHttpServer(29987); s_server->register_handler(GET, "/{FILE}", &s_test_handler); s_server->start(); @@ -130,7 +130,7 @@ TEST_F(UserFunctionCacheTest, download_normal) { // get my_add st = cache.get_function_ptr(1, "_Z6my_addv", - "http://127.0.0.1:29999/my_add.so", + "http://127.0.0.1:29987/my_add.so", my_add_md5sum, &fn_ptr, &entry); ASSERT_TRUE(st.ok()); ASSERT_TRUE(k_is_downloaded); @@ -140,7 +140,7 @@ TEST_F(UserFunctionCacheTest, download_normal) { // get my_del st = cache.get_function_ptr(1, "_Z6my_delv", - "http://127.0.0.1:29999/my_add.so", + "http://127.0.0.1:29987/my_add.so", my_add_md5sum, &fn_ptr, &entry); ASSERT_TRUE(st.ok()); ASSERT_NE(nullptr, fn_ptr); @@ -149,7 +149,7 @@ TEST_F(UserFunctionCacheTest, download_normal) { // get my_mul st = cache.get_function_ptr(1, "_Z6my_mulv", - "http://127.0.0.1:29999/my_add.so", + "http://127.0.0.1:29987/my_add.so", my_add_md5sum, &fn_ptr, &entry); ASSERT_FALSE(st.ok()); @@ -165,7 +165,7 @@ TEST_F(UserFunctionCacheTest, load_normal) { UserFunctionCacheEntry* entry = nullptr; st = cache.get_function_ptr(1, "_Z6my_addv", - "http://127.0.0.1:29999/my_add.so", + "http://127.0.0.1:29987/my_add.so", my_add_md5sum, &fn_ptr, &entry); ASSERT_TRUE(st.ok()); ASSERT_FALSE(k_is_downloaded); @@ -183,7 +183,7 @@ TEST_F(UserFunctionCacheTest, download_fail) { UserFunctionCacheEntry* entry = nullptr; st = cache.get_function_ptr(2, "_Z6my_delv", - "http://127.0.0.1:29999/my_del.so", + "http://127.0.0.1:29987/my_del.so", my_add_md5sum, &fn_ptr, &entry); ASSERT_FALSE(st.ok()); } @@ -199,7 +199,7 @@ TEST_F(UserFunctionCacheTest, md5_fail) { UserFunctionCacheEntry* entry = nullptr; st = cache.get_function_ptr(1, "_Z6my_addv", - "http://127.0.0.1:29999/my_add.so", + "http://127.0.0.1:29987/my_add.so", "1234", &fn_ptr, &entry); ASSERT_FALSE(st.ok()); } @@ -218,7 +218,7 @@ TEST_F(UserFunctionCacheTest, bad_so) { UserFunctionCacheEntry* entry = nullptr; st = cache.get_function_ptr(2, "_Z6my_addv", - "http://127.0.0.1:29999/my_add.so", + "http://127.0.0.1:29987/my_add.so", "abc", &fn_ptr, &entry); ASSERT_FALSE(st.ok()); } diff --git a/run-ut.sh b/run-ut.sh index c816e0a19e5e15..42db0a81e5bf02 100755 --- a/run-ut.sh +++ b/run-ut.sh @@ -211,6 +211,7 @@ ${DORIS_TEST_BINARY_DIR}/runtime/snapshot_loader_test ${DORIS_TEST_BINARY_DIR}/runtime/user_function_cache_test ${DORIS_TEST_BINARY_DIR}/runtime/small_file_mgr_test ${DORIS_TEST_BINARY_DIR}/runtime/mem_pool_test +${DORIS_TEST_BINARY_DIR}/runtime/memtable_flush_executor_test ${DORIS_TEST_BINARY_DIR}/runtime/memory/chunk_allocator_test ${DORIS_TEST_BINARY_DIR}/runtime/memory/system_allocator_test # Running expr Unittest From ffb37300ec3dd2da2faad43bdc24b393c61a4fb3 Mon Sep 17 00:00:00 2001 From: chenmingyu Date: Mon, 23 Sep 2019 18:42:45 +0800 Subject: [PATCH 4/8] fix by review --- be/src/olap/CMakeLists.txt | 1 + be/src/olap/delta_writer.cpp | 92 ++++------ be/src/olap/delta_writer.h | 55 ++---- be/src/olap/memtable.cpp | 15 +- be/src/olap/memtable.h | 14 +- .../memtable_flush_executor.cpp | 105 ++++++++---- be/src/olap/memtable_flush_executor.h | 158 ++++++++++++++++++ be/src/olap/storage_engine.cpp | 7 + be/src/olap/storage_engine.h | 7 +- be/src/runtime/CMakeLists.txt | 1 - be/src/runtime/exec_env.h | 3 - be/src/runtime/exec_env_init.cpp | 3 - be/src/runtime/memtable_flush_executor.h | 85 ---------- be/src/runtime/tablet_writer_mgr.cpp | 2 +- be/src/runtime/tablets_channel.cpp | 14 +- be/src/runtime/tablets_channel.h | 6 +- be/src/service/doris_main.cpp | 2 - be/test/olap/CMakeLists.txt | 1 + be/test/olap/delta_writer_test.cpp | 19 +-- .../memtable_flush_executor_test.cpp | 49 +++--- be/test/runtime/CMakeLists.txt | 1 - be/test/runtime/tablet_writer_mgr_test.cpp | 12 +- run-ut.sh | 2 +- 23 files changed, 366 insertions(+), 288 deletions(-) rename be/src/{runtime => olap}/memtable_flush_executor.cpp (53%) create mode 100644 be/src/olap/memtable_flush_executor.h delete mode 100644 be/src/runtime/memtable_flush_executor.h rename be/test/{runtime => olap}/memtable_flush_executor_test.cpp (73%) diff --git a/be/src/olap/CMakeLists.txt b/be/src/olap/CMakeLists.txt index 2ae5cc895aaa6c..a6ae95e68d2948 100644 --- a/be/src/olap/CMakeLists.txt +++ b/be/src/olap/CMakeLists.txt @@ -45,6 +45,7 @@ add_library(Olap STATIC key_coder.cpp lru_cache.cpp memtable.cpp + memtable_flush_executor.cpp merger.cpp new_status.cpp null_predicate.cpp diff --git a/be/src/olap/delta_writer.cpp b/be/src/olap/delta_writer.cpp index 55db72d00c5704..67261c95c7499f 100644 --- a/be/src/olap/delta_writer.cpp +++ b/be/src/olap/delta_writer.cpp @@ -17,34 +17,30 @@ #include "olap/delta_writer.h" -#include "olap/schema.h" -#include "olap/memtable.h" #include "olap/data_dir.h" +#include "olap/memtable.h" +#include "olap/memtable_flush_executor.h" #include "olap/rowset/rowset_factory.h" #include "olap/rowset/rowset_meta_manager.h" #include "olap/rowset/rowset_id_generator.h" -#include "runtime/memtable_flush_executor.h" +#include "olap/schema.h" +#include "olap/storage_engine.h" namespace doris { -OLAPStatus DeltaWriter::open( - WriteRequest* req, - MemTableFlushExecutor* flush_executor, - DeltaWriter** writer) { - *writer = new DeltaWriter(req, flush_executor); - return OLAP_SUCCESS; +OLAPStatus DeltaWriter::open(WriteRequest* req, DeltaWriter** writer) { + *writer = new DeltaWriter(req, StorageEngine::instance()); + return (*writer)->init(); } DeltaWriter::DeltaWriter( WriteRequest* req, - MemTableFlushExecutor* flush_executor) + StorageEngine* storage_engine) : _req(*req), _tablet(nullptr), _cur_rowset(nullptr), _new_rowset(nullptr), _new_tablet(nullptr), _rowset_writer(nullptr), _schema(nullptr), _tablet_schema(nullptr), - _delta_written_success(false), _flush_status(OLAP_SUCCESS), - _flush_executor(flush_executor) { - - _mem_table.reset(); + _delta_written_success(false), + _storage_engine(storage_engine) { } DeltaWriter::~DeltaWriter() { @@ -57,32 +53,31 @@ DeltaWriter::~DeltaWriter() { if (_rowset_writer != nullptr) { _rowset_writer->data_dir()->remove_pending_ids(ROWSET_ID_PREFIX + _rowset_writer->rowset_id().to_string()); } - LOG(INFO) << "deconstruct delta writer"; } void DeltaWriter::_garbage_collection() { OLAPStatus rollback_status = OLAP_SUCCESS; if (_tablet != nullptr) { - rollback_status = StorageEngine::instance()->txn_manager()->rollback_txn(_req.partition_id, + rollback_status = _storage_engine->txn_manager()->rollback_txn(_req.partition_id, _req.txn_id,_req.tablet_id, _req.schema_hash, _tablet->tablet_uid()); } // has to check rollback status, because the rowset maybe committed in this thread and // published in another thread, then rollback will failed // when rollback failed should not delete rowset if (rollback_status == OLAP_SUCCESS) { - StorageEngine::instance()->add_unused_rowset(_cur_rowset); + _storage_engine->add_unused_rowset(_cur_rowset); } if (_new_tablet != nullptr) { - rollback_status = StorageEngine::instance()->txn_manager()->rollback_txn(_req.partition_id, _req.txn_id, + rollback_status = _storage_engine->txn_manager()->rollback_txn(_req.partition_id, _req.txn_id, _new_tablet->tablet_id(), _new_tablet->schema_hash(), _new_tablet->tablet_uid()); if (rollback_status == OLAP_SUCCESS) { - StorageEngine::instance()->add_unused_rowset(_new_rowset); + _storage_engine->add_unused_rowset(_new_rowset); } } } OLAPStatus DeltaWriter::init() { - _tablet = StorageEngine::instance()->tablet_manager()->get_tablet(_req.tablet_id, _req.schema_hash); + _tablet = _storage_engine->tablet_manager()->get_tablet(_req.tablet_id, _req.schema_hash); if (_tablet == nullptr) { LOG(WARNING) << "tablet_id: " << _req.tablet_id << ", " << "schema_hash: " << _req.schema_hash << " not found"; @@ -95,7 +90,7 @@ OLAPStatus DeltaWriter::init() { return OLAP_ERR_RWLOCK_ERROR; } MutexLock push_lock(_tablet->get_push_lock()); - RETURN_NOT_OK(StorageEngine::instance()->txn_manager()->prepare_txn( + RETURN_NOT_OK(_storage_engine->txn_manager()->prepare_txn( _req.partition_id, _req.txn_id, _req.tablet_id, _req.schema_hash, _tablet->tablet_uid(), _req.load_id)); if (_req.need_gen_rollup) { @@ -108,7 +103,7 @@ OLAPStatus DeltaWriter::init() { << "new_tablet_id: " << new_tablet_id << ", " << "new_schema_hash: " << new_schema_hash << ", " << "transaction_id: " << _req.txn_id; - _new_tablet = StorageEngine::instance()->tablet_manager()->get_tablet(new_tablet_id, new_schema_hash); + _new_tablet = _storage_engine->tablet_manager()->get_tablet(new_tablet_id, new_schema_hash); if (_new_tablet == nullptr) { LOG(WARNING) << "find alter task, but could not find new tablet tablet_id: " << new_tablet_id << ", schema_hash: " << new_schema_hash; @@ -118,7 +113,7 @@ OLAPStatus DeltaWriter::init() { if (!new_migration_rlock.own_lock()) { return OLAP_ERR_RWLOCK_ERROR; } - StorageEngine::instance()->txn_manager()->prepare_txn( + _storage_engine->txn_manager()->prepare_txn( _req.partition_id, _req.txn_id, new_tablet_id, new_schema_hash, _new_tablet->tablet_uid(), _req.load_id); } @@ -126,7 +121,7 @@ OLAPStatus DeltaWriter::init() { } RowsetWriterContext writer_context; - writer_context.rowset_id = StorageEngine::instance()->next_rowset_id(); + writer_context.rowset_id = _storage_engine->next_rowset_id(); writer_context.tablet_uid = _tablet->tablet_uid(); writer_context.tablet_id = _req.tablet_id; writer_context.partition_id = _req.partition_id; @@ -142,9 +137,13 @@ OLAPStatus DeltaWriter::init() { _tablet_schema = &(_tablet->tablet_schema()); _schema = new Schema(*_tablet_schema); - _mem_table = std::make_shared(_tablet->tablet_id(), _schema, _tablet_schema, _req.slots, _req.tuple_desc, _tablet->keys_type()); + _mem_table = std::make_shared(_tablet->tablet_id(), _schema, _tablet_schema, _req.slots, + _req.tuple_desc, _tablet->keys_type(), _rowset_writer.get()); - _flush_queue_idx = _flush_executor->get_queue_idx(_tablet->data_dir()->path_hash()); + // create flush handler + FlushHandler* flush_handler; + RETURN_NOT_OK(_storage_engine->memtable_flush_executor()->create_flush_handler(_tablet->data_dir()->path_hash(), &flush_handler)); + _flush_handler.reset(flush_handler); _is_init = true; return OLAP_SUCCESS; @@ -152,10 +151,7 @@ OLAPStatus DeltaWriter::init() { OLAPStatus DeltaWriter::write(Tuple* tuple) { if (!_is_init) { - auto st = init(); - if (st != OLAP_SUCCESS) { - return st; - } + RETURN_NOT_OK(init()); } _mem_table->insert(tuple); @@ -165,42 +161,28 @@ OLAPStatus DeltaWriter::write(Tuple* tuple) { if (_mem_table->memory_usage() >= config::write_buffer_size) { RETURN_NOT_OK(_flush_memtable_async()); // create a new memtable for new incoming data - _mem_table.reset(new MemTable(_tablet->tablet_id(), _schema, _tablet_schema, _req.slots, _req.tuple_desc, _tablet->keys_type())); + _mem_table.reset(new MemTable(_tablet->tablet_id(), _schema, _tablet_schema, _req.slots, + _req.tuple_desc, _tablet->keys_type(), _rowset_writer.get())); } return OLAP_SUCCESS; } OLAPStatus DeltaWriter::_flush_memtable_async() { - // last flush already failed, return error - RETURN_NOT_OK(_flush_status.load()); - - MemTableFlushContext ctx; - ctx.memtable = _mem_table; - ctx.delta_writer = this; - ctx.flush_status = &_flush_status; - _flush_future = _flush_executor->push_memtable(_flush_queue_idx, ctx); - return OLAP_SUCCESS; + return _flush_handler->submit(_mem_table); } -OLAPStatus DeltaWriter::flush() { +OLAPStatus DeltaWriter::close() { if (!_is_init) { - auto st = init(); - if (st != OLAP_SUCCESS) { - return st; - } + RETURN_NOT_OK(init()); } RETURN_NOT_OK(_flush_memtable_async()); return OLAP_SUCCESS; } -OLAPStatus DeltaWriter::close(google::protobuf::RepeatedPtrField* tablet_vec) { +OLAPStatus DeltaWriter::close_wait(google::protobuf::RepeatedPtrField* tablet_vec) { // return error if previous flush failed - RETURN_NOT_OK(_flush_status.load()); - - // wait for the last memtable flushed - // flush() is called before close(), so _flush_future is set at least once. - RETURN_NOT_OK(_flush_future.get()); + RETURN_NOT_OK(_flush_handler->wait()); // use rowset meta manager to save meta _cur_rowset = _rowset_writer->build(); @@ -208,7 +190,7 @@ OLAPStatus DeltaWriter::close(google::protobuf::RepeatedPtrField* t LOG(WARNING) << "fail to build rowset"; return OLAP_ERR_MALLOC_ERROR; } - OLAPStatus res = StorageEngine::instance()->txn_manager()->commit_txn(_tablet->data_dir()->get_meta(), + OLAPStatus res = _storage_engine->txn_manager()->commit_txn(_tablet->data_dir()->get_meta(), _req.partition_id, _req.txn_id,_req.tablet_id, _req.schema_hash, _tablet->tablet_uid(), _req.load_id, _cur_rowset, false); if (res != OLAP_SUCCESS && res != OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST) { @@ -229,7 +211,7 @@ OLAPStatus DeltaWriter::close(google::protobuf::RepeatedPtrField* t return res; } - res = StorageEngine::instance()->txn_manager()->commit_txn(_new_tablet->data_dir()->get_meta(), + res = _storage_engine->txn_manager()->commit_txn(_new_tablet->data_dir()->get_meta(), _req.partition_id, _req.txn_id, _new_tablet->tablet_id(), _new_tablet->schema_hash(), _new_tablet->tablet_uid(), _req.load_id, _new_rowset, false); @@ -254,9 +236,9 @@ OLAPStatus DeltaWriter::close(google::protobuf::RepeatedPtrField* t _delta_written_success = true; + const FlushStatistic& stat = _flush_handler->get_stats(); LOG(INFO) << "close delta writer for tablet: " << _tablet->tablet_id() - << ", flush time(ms): " << _flush_time_ns / 1000 / 1000 - << ", flush count: " << _flush_count; + << ", stats: " << stat; return OLAP_SUCCESS; } diff --git a/be/src/olap/delta_writer.h b/be/src/olap/delta_writer.h index 8a76ceaf5ede5f..073f5e5f1ea236 100644 --- a/be/src/olap/delta_writer.h +++ b/be/src/olap/delta_writer.h @@ -18,7 +18,6 @@ #ifndef DORIS_BE_SRC_DELTA_WRITER_H #define DORIS_BE_SRC_DELTA_WRITER_H -#include "olap/storage_engine.h" #include "olap/tablet.h" #include "olap/schema_change.h" #include "runtime/descriptors.h" @@ -29,10 +28,11 @@ namespace doris { -class SegmentGroup; +class FlushHandler; class MemTable; -class MemTableFlushExecutor; class Schema; +class SegmentGroup; +class StorageEngine; enum WriteType { LOAD = 1, @@ -55,37 +55,24 @@ struct WriteRequest { class DeltaWriter { public: - static OLAPStatus open( - WriteRequest* req, - MemTableFlushExecutor* _flush_executor, - DeltaWriter** writer); + static OLAPStatus open(WriteRequest* req, DeltaWriter** writer); - OLAPStatus init(); + DeltaWriter(WriteRequest* req, StorageEngine* storage_engine); - DeltaWriter(WriteRequest* req, MemTableFlushExecutor* _flush_executor); + OLAPStatus init(); ~DeltaWriter(); OLAPStatus write(Tuple* tuple); - // flush the last memtable to flush queue, must call it before close - OLAPStatus flush(); - - - - OLAPStatus close(google::protobuf::RepeatedPtrField* tablet_vec); + // flush the last memtable to flush queue, must call it before close_wait() + OLAPStatus close(); + // wait for all memtables being flushed + OLAPStatus close_wait(google::protobuf::RepeatedPtrField* tablet_vec); OLAPStatus cancel(); int64_t partition_id() const { return _req.partition_id; } - OLAPStatus get_flush_status() { return _flush_status.load(); } - RowsetWriter* rowset_writer() { return _rowset_writer.get(); } - - void update_flush_time(int64_t flush_ns) { - _flush_time_ns += flush_ns; - _flush_count++; - } - private: // push a full memtable to flush executor OLAPStatus _flush_memtable_async(); @@ -105,26 +92,8 @@ class DeltaWriter { const TabletSchema* _tablet_schema; bool _delta_written_success; - // the flush status of previous memtable. - // the default is OLAP_SUCCESS, and once it changes to some ERROR code, - // it will never change back to OLAP_SUCCESS. - // this status will be checked each time the next memtable is going to be flushed, - // so that if the previous flush is already failed, no need to flush next memtable. - std::atomic _flush_status; - // the future of the very last memtable flush execution. - // because the flush of this delta writer's memtables are executed serially, - // if the last memtable is flushed, all previous memtables should already be flushed. - // so we only need to wait and block on the last memtable's flush future. - std::future _flush_future; - // total flush time and flush count of memtables - int64_t _flush_time_ns; - int64_t _flush_count; - - MemTableFlushExecutor* _flush_executor; - // the idx of flush queues vector in MemTableFlushExecutor. - // this idx is got from MemTableFlushExecutor, - // and memtables of this delta writer will be pushed to this certain flush queue only. - int32_t _flush_queue_idx; + StorageEngine* _storage_engine; + std::shared_ptr _flush_handler; }; } // namespace doris diff --git a/be/src/olap/memtable.cpp b/be/src/olap/memtable.cpp index 761689e340613c..5a1b240ae219b9 100644 --- a/be/src/olap/memtable.cpp +++ b/be/src/olap/memtable.cpp @@ -28,14 +28,15 @@ namespace doris { MemTable::MemTable(int64_t tablet_id, Schema* schema, const TabletSchema* tablet_schema, const std::vector* slot_descs, TupleDescriptor* tuple_desc, - KeysType keys_type) + KeysType keys_type, RowsetWriter* rowset_writer) : _tablet_id(tablet_id), _schema(schema), _tablet_schema(tablet_schema), _tuple_desc(tuple_desc), _slot_descs(slot_descs), _keys_type(keys_type), - _row_comparator(_schema) { + _row_comparator(_schema), + _rowset_writer(rowset_writer) { _schema_size = _schema->schema_size(); _tuple_buf = _arena.Allocate(_schema_size); _skip_list = new Table(_row_comparator, &_arena); @@ -77,7 +78,7 @@ void MemTable::insert(Tuple* tuple) { } } -OLAPStatus MemTable::flush(RowsetWriter* rowset_writer) { +OLAPStatus MemTable::flush() { int64_t duration_ns = 0; { SCOPED_RAW_TIMER(&duration_ns); @@ -86,17 +87,17 @@ OLAPStatus MemTable::flush(RowsetWriter* rowset_writer) { char* row = (char*)it.key(); ContiguousRow dst_row(_schema, row); agg_finalize_row(&dst_row, _skip_list->arena()); - RETURN_NOT_OK(rowset_writer->add_row(dst_row)); + RETURN_NOT_OK(_rowset_writer->add_row(dst_row)); } - RETURN_NOT_OK(rowset_writer->flush()); + RETURN_NOT_OK(_rowset_writer->flush()); } DorisMetrics::memtable_flush_total.increment(1); DorisMetrics::memtable_flush_duration_us.increment(duration_ns / 1000); return OLAP_SUCCESS; } -OLAPStatus MemTable::close(RowsetWriter* rowset_writer) { - return flush(rowset_writer); +OLAPStatus MemTable::close() { + return flush(); } } // namespace doris diff --git a/be/src/olap/memtable.h b/be/src/olap/memtable.h index 76f05b11c0ad17..5fb428e03f471f 100644 --- a/be/src/olap/memtable.h +++ b/be/src/olap/memtable.h @@ -29,21 +29,19 @@ namespace doris { class RowCursor; +class RowsetWriter; class MemTable { public: MemTable(int64_t tablet_id, Schema* schema, const TabletSchema* tablet_schema, const std::vector* slot_descs, TupleDescriptor* tuple_desc, - KeysType keys_type); + KeysType keys_type, RowsetWriter* rowset_writer); ~MemTable(); int64_t tablet_id() { return _tablet_id; } size_t memory_usage(); void insert(Tuple* tuple); - OLAPStatus flush(RowsetWriter* rowset_writer); - OLAPStatus close(RowsetWriter* rowset_writer); - - std::future get_flush_future() { return _flush_promise.get_future(); } - void set_flush_status(OLAPStatus st) { _flush_promise.set_value(st); } + OLAPStatus flush(); + OLAPStatus close(); private: int64_t _tablet_id; @@ -68,8 +66,8 @@ class MemTable { size_t _schema_size; Table* _skip_list; - // the promise it to save result status of flush - std::promise _flush_promise; + RowsetWriter* _rowset_writer; + }; // class MemTable } // namespace doris diff --git a/be/src/runtime/memtable_flush_executor.cpp b/be/src/olap/memtable_flush_executor.cpp similarity index 53% rename from be/src/runtime/memtable_flush_executor.cpp rename to be/src/olap/memtable_flush_executor.cpp index 256dc39c76fe11..90239d6fef38a1 100644 --- a/be/src/runtime/memtable_flush_executor.cpp +++ b/be/src/olap/memtable_flush_executor.cpp @@ -15,28 +15,64 @@ // specific language governing permissions and limitations // under the License. -#include "runtime/memtable_flush_executor.h" +#include "olap/memtable_flush_executor.h" -#include "olap/memtable.h" +#include "olap/data_dir.h" #include "olap/delta_writer.h" +#include "olap/memtable.h" #include "runtime/exec_env.h" namespace doris { -MemTableFlushExecutor::MemTableFlushExecutor(ExecEnv* exec_env): - _exec_env(exec_env) { +OLAPStatus FlushHandler::submit(std::shared_ptr memtable) { + RETURN_NOT_OK(_last_flush_status.load()); + MemTableFlushContext ctx; + ctx.memtable = memtable; + ctx.flush_handler = this->shared_from_this(); + _flush_futures.push(_flush_executor->_push_memtable(_flush_queue_idx, ctx)); + return OLAP_SUCCESS; } -void MemTableFlushExecutor::init() { -#ifndef BE_TEST - int32_t data_dir_num = _exec_env->store_paths().size(); - _thread_num_per_store = std::max(1, config::flush_thread_num_per_store); - _num_threads = data_dir_num * _thread_num_per_store; -#else - int32_t data_dir_num = 1; +OLAPStatus FlushHandler::wait() { + if (_last_flush_status.load() != OLAP_SUCCESS) { + return _last_flush_status.load(); + } + + while(!_flush_futures.empty()) { + std::future& fu = _flush_futures.front(); + OLAPStatus st = fu.get(); + if (st != OLAP_SUCCESS) { + _last_flush_status.store(st); + return st; + } + _flush_futures.pop(); + } + return OLAP_SUCCESS; +} + +const FlushStatistic& FlushHandler::get_stats() { + return _stats; +} + +void FlushHandler::on_flush_finished(const FlushResult& res) { + if (res.flush_status != OLAP_SUCCESS) { + _last_flush_status.store(res.flush_status); + } else { + _stats.flush_time_ns.fetch_add(res.flush_time_ns); + _stats.flush_count.fetch_add(1); + } +} + +OLAPStatus MemTableFlushExecutor::create_flush_handler(int64_t path_hash, FlushHandler** flush_handler) { + int32_t flush_queue_idx = _get_queue_idx(path_hash); + *flush_handler = new FlushHandler(flush_queue_idx, this); + return OLAP_SUCCESS; +} + +void MemTableFlushExecutor::init(const std::vector& data_dirs) { + int32_t data_dir_num = data_dirs.size(); _thread_num_per_store = std::max(1, config::flush_thread_num_per_store); _num_threads = data_dir_num * _thread_num_per_store; -#endif // create flush queues for (int i = 0; i < _num_threads; ++i) { @@ -55,7 +91,7 @@ void MemTableFlushExecutor::init() { // so there are 8(= 4 * 2) queues in _flush_queues. // and the path hash of the 4 paths are mapped to idx 0, 2, 4, 6. int32_t group = 0; - for (auto store : _exec_env->storage_engine()->get_stores()) { + for (auto store : data_dirs) { _path_map[store->path_hash()] = group; group += _thread_num_per_store; } @@ -80,7 +116,7 @@ MemTableFlushExecutor::~MemTableFlushExecutor() { delete _flush_pool; } -int32_t MemTableFlushExecutor::get_queue_idx(size_t path_hash) { +int32_t MemTableFlushExecutor::_get_queue_idx(size_t path_hash) { std::lock_guard l(_lock); int32_t cur_idx = _path_map[path_hash]; int32_t group = cur_idx / _thread_num_per_store; @@ -90,9 +126,17 @@ int32_t MemTableFlushExecutor::get_queue_idx(size_t path_hash) { return cur_idx; } -std::future MemTableFlushExecutor::push_memtable(int32_t queue_idx, const MemTableFlushContext& ctx) { +std::future MemTableFlushExecutor::_push_memtable(int32_t queue_idx, MemTableFlushContext& ctx) { + ctx.flush_id = _id_generator.fetch_add(1); + std::promise promise; + std::future fu = promise.get_future(); + { + std::lock_guard l(_lock); + _flush_promises[ctx.flush_id] = std::move(promise); + } + _flush_queues[queue_idx]->blocking_put(ctx); - return ctx.memtable->get_flush_future(); + return fu; } void MemTableFlushExecutor::_flush_memtable(int32_t queue_idx) { @@ -103,26 +147,31 @@ void MemTableFlushExecutor::_flush_memtable(int32_t queue_idx) { return; } - DeltaWriter* delta_writer = ctx.delta_writer; // if last flush of this tablet already failed, just skip - if (delta_writer->get_flush_status() != OLAP_SUCCESS) { + if (ctx.flush_handler->last_flush_status() != OLAP_SUCCESS) { continue; } // flush the memtable + FlushResult res; MonotonicStopWatch timer; timer.start(); - OLAPStatus st = ctx.memtable->flush(delta_writer->rowset_writer()); - if (st == OLAP_SUCCESS) { - delta_writer->update_flush_time(timer.elapsed_time()); - } else { - // if failed, update the flush status, this staus will be observed - // by delta writer. - ctx.flush_status->store(st); + res.flush_status = ctx.memtable->flush(); + res.flush_time_ns = timer.elapsed_time(); + ctx.flush_handler->on_flush_finished(res); + + { + std::lock_guard l(_lock); + _flush_promises[ctx.flush_id].set_value(res.flush_status); + _flush_promises.erase(ctx.flush_id); } - // set the promise's value - ctx.memtable->set_flush_status(st); } } -} // end of namespace +std::ostream& operator<<(std::ostream& os, const FlushStatistic& stat) { + os << "(flush time(ms)=" << stat.flush_time_ns / 1000 / 1000 + << ", flush count=" << stat.flush_count << ")"; + return os; +} + +} // end of namespac diff --git a/be/src/olap/memtable_flush_executor.h b/be/src/olap/memtable_flush_executor.h new file mode 100644 index 00000000000000..cf7279cd26a28c --- /dev/null +++ b/be/src/olap/memtable_flush_executor.h @@ -0,0 +1,158 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + +#include "olap/olap_define.h" +#include "util/blocking_queue.hpp" +#include "util/spinlock.h" +#include "util/thread_pool.hpp" + +namespace doris { + +class DataDir; +class DeltaWriter; +class ExecEnv; +class MemTable; + +// The context for a memtable to be flushed. +class FlushHandler; +struct MemTableFlushContext { + // generated by memtable flush executor + // denote a certain memtable flush execution. + int64_t flush_id; + // memtable to be flushed + std::shared_ptr memtable; + // flush handler from a delta writer. + // use shared ptr because flush_handler may be deleted before this + // memtable being flushed. so we need to make sure the flush_handler + // is alive until this memtable being flushed. + std::shared_ptr flush_handler; +}; + +// the flush result of a single memtable flush +struct FlushResult { + OLAPStatus flush_status; + int64_t flush_time_ns; +}; + +// the statistic of a certain flush handler. +// use atomic because it may be updated by multi threads +struct FlushStatistic { + std::atomic_int64_t flush_time_ns = {0}; + std::atomic_int64_t flush_count= {0}; +}; +std::ostream& operator<<(std::ostream& os, const FlushStatistic& stat); + +// flush handler is for flushing memtables in a delta writer +// This class must be wrapped by std::shared_ptr, or you will get bad_weak_ptr exception +// when calling submit(); +class MemTableFlushExecutor; +class FlushHandler : public std::enable_shared_from_this { +public: + FlushHandler(int32_t flush_queue_idx, MemTableFlushExecutor* flush_executor): + _flush_queue_idx(flush_queue_idx), + _last_flush_status(OLAP_SUCCESS), + _flush_executor(flush_executor) { + } + + // submit a memtable to flush. return error if some previous submitted MemTable has failed + OLAPStatus submit(std::shared_ptr memtable); + // wait for all submitted memtable finished. + OLAPStatus wait(); + // get flush operations' statistics + const FlushStatistic& get_stats(); + // called when a memtable is finished by executor. + void on_flush_finished(const FlushResult& res); + + OLAPStatus last_flush_status() { return _last_flush_status.load(); } +private: + // flush queue idx in memtable flush executor + int32_t _flush_queue_idx; + // the flush status of last memtable + std::atomic _last_flush_status; + // save all futures of memtable flush execution. + std::queue> _flush_futures; + FlushStatistic _stats; + MemTableFlushExecutor* _flush_executor; +}; + +// MemTableFlushExecutor is for flushing memtables to disk. +// Each data directory has a specified number of worker threads and a corresponding number of flush queues. +// Each worker thread only takes memtable from the corresponding flush queue and writes it to disk. +// User SHOULD NOT call method of this class directly, use pattern should be: +// +// ... +// FlushHandler* flush_handler; +// memTableFlushExecutor.create_flush_handler(path_hash, &flush_handler); +// std::shared_ptr shared_handler(flush_handler); +// ... +// flush_handler->submit(memtable) +// ... +class MemTableFlushExecutor { +public: + MemTableFlushExecutor() {} + // init should be called after storage engine is opened, + // because it needs path hash of each data dir. + void init(const std::vector& data_dirs); + + ~MemTableFlushExecutor(); + + // create a flush handler to access the flush executor + OLAPStatus create_flush_handler(int64_t path_hash, FlushHandler** flush_handler); + +private: + + // given the path hash, return the next idx of flush queue. + // eg. + // path A is mapped to idx 0 and 1, so each time get_queue_idx(A) is called, + // 0 and 1 will returned alternately. + int32_t _get_queue_idx(size_t path_hash); + + // push the memtable to specified flush queue, and return a future + std::future _push_memtable(int32_t queue_idx, MemTableFlushContext& ctx); + + void _flush_memtable(int32_t queue_idx); + +private: + friend class FlushHandler; + + int32_t _thread_num_per_store; + int32_t _num_threads; + ThreadPool* _flush_pool; + // the size of this vector should equals to _num_threads + std::vector*> _flush_queues; + + std::atomic_int64_t _id_generator; + + // lock to protect path_map + SpinLock _lock; + // path hash -> queue idx of _flush_queues; + std::unordered_map _path_map; + std::unordered_map> _flush_promises; +}; + +} // end namespace diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp index 11ed82d604aa19..1f670d01cd6d5e 100644 --- a/be/src/olap/storage_engine.cpp +++ b/be/src/olap/storage_engine.cpp @@ -36,6 +36,7 @@ #include "olap/base_compaction.h" #include "olap/cumulative_compaction.h" #include "olap/lru_cache.h" +#include "olap/memtable_flush_executor.h" #include "olap/tablet_meta.h" #include "olap/tablet_meta_manager.h" #include "olap/push_handler.h" @@ -222,6 +223,9 @@ OLAPStatus StorageEngine::open() { // 取消未完成的SchemaChange任务 _tablet_manager->cancel_unfinished_schema_change(); + _memtable_flush_executor = new MemTableFlushExecutor(); + _memtable_flush_executor->init(dirs); + return OLAP_SUCCESS; } @@ -492,6 +496,9 @@ OLAPStatus StorageEngine::clear() { store_pair.second = nullptr; } _store_map.clear(); + + delete _memtable_flush_executor; + return OLAP_SUCCESS; } diff --git a/be/src/olap/storage_engine.h b/be/src/olap/storage_engine.h index cfbd995b24615a..7f96fa4bb05072 100644 --- a/be/src/olap/storage_engine.h +++ b/be/src/olap/storage_engine.h @@ -51,9 +51,10 @@ namespace doris { -class Tablet; class DataDir; class EngineTask; +class MemTableFlushExecutor; +class Tablet; // StorageEngine singleton to manage all Table pointers. // Providing add/drop/get operations. @@ -203,6 +204,8 @@ class StorageEngine { void release_rowset_id(const RowsetId& rowset_id) { return _rowset_id_generator->release_id(rowset_id); }; + MemTableFlushExecutor* memtable_flush_executor() { return _memtable_flush_executor; } + private: OLAPStatus _check_file_descriptor_number(); @@ -345,6 +348,8 @@ class StorageEngine { std::unique_ptr _rowset_id_generator; + MemTableFlushExecutor* _memtable_flush_executor; + DISALLOW_COPY_AND_ASSIGN(StorageEngine); }; diff --git a/be/src/runtime/CMakeLists.txt b/be/src/runtime/CMakeLists.txt index d38fd7254b790d..ae89fca1199998 100644 --- a/be/src/runtime/CMakeLists.txt +++ b/be/src/runtime/CMakeLists.txt @@ -84,7 +84,6 @@ set(RUNTIME_FILES export_sink.cpp tablet_writer_mgr.cpp tablets_channel.cpp - memtable_flush_executor.cpp bufferpool/buffer_allocator.cc bufferpool/buffer_pool.cc bufferpool/reservation_tracker.cc diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 3cfd3a4bb87bb6..b14be057904f5d 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -37,7 +37,6 @@ class FragmentMgr; class LoadPathMgr; class LoadStreamMgr; class MemTracker; -class MemTableFlushExecutor; class MetricRegistry; class StorageEngine; class PoolMemTrackerRegistry; @@ -127,7 +126,6 @@ class ExecEnv { StreamLoadExecutor* stream_load_executor() { return _stream_load_executor; } RoutineLoadTaskExecutor* routine_load_task_executor() { return _routine_load_task_executor; } - MemTableFlushExecutor* memtable_flush_executor() { return _memtable_flush_executor; } private: Status _init(const std::vector& store_paths); @@ -178,7 +176,6 @@ class ExecEnv { StreamLoadExecutor* _stream_load_executor = nullptr; RoutineLoadTaskExecutor* _routine_load_task_executor = nullptr; SmallFileMgr* _small_file_mgr = nullptr; - MemTableFlushExecutor* _memtable_flush_executor = nullptr; }; } diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 63789093a80c4e..87184a04b68fb4 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -30,7 +30,6 @@ #include "runtime/result_buffer_mgr.h" #include "runtime/result_queue_mgr.h" #include "runtime/mem_tracker.h" -#include "runtime/memtable_flush_executor.h" #include "runtime/thread_resource_mgr.h" #include "runtime/fragment_mgr.h" #include "runtime/tablet_writer_mgr.h" @@ -106,7 +105,6 @@ Status ExecEnv::_init(const std::vector& store_paths) { _stream_load_executor = new StreamLoadExecutor(this); _routine_load_task_executor = new RoutineLoadTaskExecutor(this); _small_file_mgr = new SmallFileMgr(this, config::small_file_dir); - _memtable_flush_executor = new MemTableFlushExecutor(this); _backend_client_cache->init_metrics(DorisMetrics::metrics(), "backend"); _frontend_client_cache->init_metrics(DorisMetrics::metrics(), "frontend"); @@ -238,7 +236,6 @@ void ExecEnv::_destory() { delete _stream_load_executor; delete _routine_load_task_executor; delete _external_scan_context_mgr; - delete _memtable_flush_executor; _metrics = nullptr; } diff --git a/be/src/runtime/memtable_flush_executor.h b/be/src/runtime/memtable_flush_executor.h deleted file mode 100644 index 10dca477e7c294..00000000000000 --- a/be/src/runtime/memtable_flush_executor.h +++ /dev/null @@ -1,85 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include -#include -#include -#include -#include -#include -#include - -#include "olap/olap_define.h" -#include "util/blocking_queue.hpp" -#include "util/spinlock.h" -#include "util/thread_pool.hpp" - -namespace doris { - -class ExecEnv; -class DeltaWriter; -class MemTable; - -// The context for a memtable to be flushed. -// It does not own any objects in it. -struct MemTableFlushContext { - std::shared_ptr memtable; - DeltaWriter* delta_writer; - std::atomic* flush_status; -}; - -// MemTableFlushExecutor is for flushing memtables to disk. -// Each data directory has a specified number of worker threads and a corresponding number of flush queues. -// Each worker thread only takes memtable from the corresponding flush queue and writes it to disk. -class MemTableFlushExecutor { -public: - MemTableFlushExecutor(ExecEnv* exec_env); - // init should be called after storage engine is opened, - // because it needs path hash of each data dir. - void init(); - - ~MemTableFlushExecutor(); - - // given the path hash, return the next idx of flush queue. - // eg. - // path A is mapped to idx 0 and 1, so each time get_queue_idx(A) is called, - // 0 and 1 will returned alternately. - int32_t get_queue_idx(size_t path_hash); - - // push the memtable to specified flush queue, and return a future - std::future push_memtable(int32_t queue_idx, const MemTableFlushContext& ctx); - -private: - void _flush_memtable(int32_t queue_idx); - -private: - ExecEnv* _exec_env; - int32_t _thread_num_per_store; - int32_t _num_threads; - ThreadPool* _flush_pool; - // the size of this vector should equals to _num_threads - std::vector*> _flush_queues; - - // lock to protect path_map - SpinLock _lock; - // path hash -> queue idx of _flush_queues; - std::unordered_map _path_map; -}; - -} // end namespace diff --git a/be/src/runtime/tablet_writer_mgr.cpp b/be/src/runtime/tablet_writer_mgr.cpp index 498b2d1f25e9f6..20593ce1927709 100644 --- a/be/src/runtime/tablet_writer_mgr.cpp +++ b/be/src/runtime/tablet_writer_mgr.cpp @@ -54,7 +54,7 @@ Status TabletWriterMgr::open(const PTabletWriterOpenRequest& params) { channel = *val; } else { // create a new - channel.reset(new TabletsChannel(key, _exec_env->memtable_flush_executor())); + channel.reset(new TabletsChannel(key)); _tablets_channels.insert(key, channel); } } diff --git a/be/src/runtime/tablets_channel.cpp b/be/src/runtime/tablets_channel.cpp index 3333b337208f4f..5a299fcf4ab278 100644 --- a/be/src/runtime/tablets_channel.cpp +++ b/be/src/runtime/tablets_channel.cpp @@ -25,8 +25,8 @@ namespace doris { -TabletsChannel::TabletsChannel(const TabletsChannelKey& key, MemTableFlushExecutor* flush_executor): - _key(key), _flush_executor(flush_executor), _closed_senders(64) { +TabletsChannel::TabletsChannel(const TabletsChannelKey& key): + _key(key), _closed_senders(64) { } TabletsChannel::~TabletsChannel() { @@ -121,11 +121,11 @@ Status TabletsChannel::close(int sender_id, bool* finished, *finished = (_num_remaining_senders == 0); if (*finished) { // All senders are closed - // 1. flush all delta writers + // 1. close all delta writers std::vector need_wait_writers; for (auto& it : _tablet_writers) { if (_partition_ids.count(it.second->partition_id()) > 0) { - auto st = it.second->flush(); + auto st = it.second->close(); if (st != OLAP_SUCCESS) { LOG(WARNING) << "close tablet writer failed, tablet_id=" << it.first << ", transaction_id=" << _txn_id << ", err=" << st; @@ -144,11 +144,11 @@ Status TabletsChannel::close(int sender_id, bool* finished, } } - // 2. wait and close delta writers and build the tablet vector + // 2. wait delta writers and build the tablet vector for (auto writer : need_wait_writers) { // close may return failed, but no need to handle it here. // tablet_vec will only contains success tablet, and then let FE judge it. - writer->close(tablet_vec); + writer->close_wait(tablet_vec); } } return Status::OK(); @@ -182,7 +182,7 @@ Status TabletsChannel::_open_all_writers(const PTabletWriterOpenRequest& params) request.slots = index_slots; DeltaWriter* writer = nullptr; - auto st = DeltaWriter::open(&request, _flush_executor, &writer); + auto st = DeltaWriter::open(&request, &writer); if (st != OLAP_SUCCESS) { std::stringstream ss; ss << "open delta writer failed, tablet_id=" << tablet.tablet_id() diff --git a/be/src/runtime/tablets_channel.h b/be/src/runtime/tablets_channel.h index bd9b90b7550d3b..b893af870b02a1 100644 --- a/be/src/runtime/tablets_channel.h +++ b/be/src/runtime/tablets_channel.h @@ -24,6 +24,7 @@ #include "runtime/mem_tracker.h" #include "util/bitmap.h" #include "util/thread_pool.hpp" +#include "util/uid_util.h" #include "gen_cpp/Types_types.h" #include "gen_cpp/PaloInternalService_types.h" @@ -55,14 +56,12 @@ struct TabletsChannelKeyHasher { }; class DeltaWriter; -class MemTable; -class MemTableFlushExecutor; class OlapTableSchemaParam; // channel that process all data for this load class TabletsChannel { public: - TabletsChannel(const TabletsChannelKey& key, MemTableFlushExecutor* flush_executor); + TabletsChannel(const TabletsChannelKey& key); ~TabletsChannel(); @@ -85,7 +84,6 @@ class TabletsChannel { private: // id of this load channel TabletsChannelKey _key; - MemTableFlushExecutor* _flush_executor; // make execute sequece std::mutex _lock; diff --git a/be/src/service/doris_main.cpp b/be/src/service/doris_main.cpp index d052a582ac745c..a916039e2fa576 100644 --- a/be/src/service/doris_main.cpp +++ b/be/src/service/doris_main.cpp @@ -39,7 +39,6 @@ #include "common/status.h" #include "codegen/llvm_codegen.h" #include "runtime/exec_env.h" -#include "runtime/memtable_flush_executor.h" #include "util/logging.h" #include "util/network_util.h" #include "util/thrift_util.h" @@ -167,7 +166,6 @@ int main(int argc, char** argv) { auto exec_env = doris::ExecEnv::GetInstance(); doris::ExecEnv::init(exec_env, paths); exec_env->set_storage_engine(engine); - exec_env->memtable_flush_executor()->init(); doris::FrontendHelper::setup(exec_env); doris::ThriftServer* be_server = nullptr; diff --git a/be/test/olap/CMakeLists.txt b/be/test/olap/CMakeLists.txt index 9d671d6f30c7d1..3f248e6c3017b5 100644 --- a/be/test/olap/CMakeLists.txt +++ b/be/test/olap/CMakeLists.txt @@ -71,3 +71,4 @@ ADD_BE_TEST(key_coder_test) ADD_BE_TEST(short_key_index_test) ADD_BE_TEST(page_cache_test) ADD_BE_TEST(hll_test) +ADD_BE_TEST(memtable_flush_executor_test) diff --git a/be/test/olap/delta_writer_test.cpp b/be/test/olap/delta_writer_test.cpp index 4b10a059afa434..9685aeb3cf0322 100644 --- a/be/test/olap/delta_writer_test.cpp +++ b/be/test/olap/delta_writer_test.cpp @@ -31,7 +31,6 @@ #include "runtime/tuple.h" #include "runtime/descriptor_helper.h" #include "runtime/exec_env.h" -#include "runtime/memtable_flush_executor.h" #include "util/logging.h" #include "olap/options.h" #include "olap/tablet_meta_manager.h" @@ -45,7 +44,6 @@ static const uint32_t MAX_RETRY_TIMES = 10; static const uint32_t MAX_PATH_LEN = 1024; StorageEngine* k_engine = nullptr; -MemTableFlushExecutor* k_flush_executor = nullptr; void set_up() { char buffer[MAX_PATH_LEN]; @@ -62,16 +60,11 @@ void set_up() { ExecEnv* exec_env = doris::ExecEnv::GetInstance(); exec_env->set_storage_engine(k_engine); - - k_flush_executor = new MemTableFlushExecutor(exec_env); - k_flush_executor->init(); } void tear_down() { delete k_engine; k_engine = nullptr; - delete k_flush_executor; - k_flush_executor = nullptr; system("rm -rf ./data_test"); remove_all_dir(std::string(getenv("DORIS_HOME")) + UNUSED_PREFIX); } @@ -310,11 +303,11 @@ TEST_F(TestDeltaWriter, open) { WriteRequest write_req = {10003, 270068375, WriteType::LOAD, 20001, 30001, load_id, false, tuple_desc}; DeltaWriter* delta_writer = nullptr; - DeltaWriter::open(&write_req, k_flush_executor, &delta_writer); + DeltaWriter::open(&write_req, &delta_writer); ASSERT_NE(delta_writer, nullptr); - res = delta_writer->flush(); + res = delta_writer->close(); ASSERT_EQ(OLAP_SUCCESS, res); - res = delta_writer->close(nullptr); + res = delta_writer->close_wait(nullptr); ASSERT_EQ(OLAP_SUCCESS, res); SAFE_DELETE(delta_writer); @@ -345,7 +338,7 @@ TEST_F(TestDeltaWriter, write) { 20002, 30002, load_id, false, tuple_desc, &(tuple_desc->slots())}; DeltaWriter* delta_writer = nullptr; - DeltaWriter::open(&write_req, k_flush_executor, &delta_writer); + DeltaWriter::open(&write_req, &delta_writer); ASSERT_NE(delta_writer, nullptr); Arena arena; @@ -404,9 +397,9 @@ TEST_F(TestDeltaWriter, write) { ASSERT_EQ(OLAP_SUCCESS, res); } - res = delta_writer->flush(); + res = delta_writer->close(); ASSERT_EQ(OLAP_SUCCESS, res); - res = delta_writer->close(nullptr); + res = delta_writer->close_wait(nullptr); ASSERT_EQ(OLAP_SUCCESS, res); // publish version success diff --git a/be/test/runtime/memtable_flush_executor_test.cpp b/be/test/olap/memtable_flush_executor_test.cpp similarity index 73% rename from be/test/runtime/memtable_flush_executor_test.cpp rename to be/test/olap/memtable_flush_executor_test.cpp index 79ba2409ba3571..4427e789a854ae 100644 --- a/be/test/runtime/memtable_flush_executor_test.cpp +++ b/be/test/olap/memtable_flush_executor_test.cpp @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "runtime/memtable_flush_executor.h" +#include "olap/memtable_flush_executor.h" #include #include @@ -34,7 +34,6 @@ #include "runtime/tuple.h" #include "runtime/descriptor_helper.h" #include "runtime/exec_env.h" -#include "runtime/memtable_flush_executor.h" #include "util/logging.h" #include "olap/options.h" #include "olap/tablet_meta_manager.h" @@ -47,7 +46,7 @@ MemTableFlushExecutor* k_flush_executor = nullptr; void set_up() { char buffer[1024]; getcwd(buffer, 1024); - config::storage_root_path = std::string(buffer) + "/data_test"; + config::storage_root_path = std::string(buffer) + "/flush_test"; remove_all_dir(config::storage_root_path); create_dir(config::storage_root_path); std::vector paths; @@ -60,16 +59,13 @@ void set_up() { ExecEnv* exec_env = doris::ExecEnv::GetInstance(); exec_env->set_storage_engine(k_engine); - k_flush_executor = new MemTableFlushExecutor(exec_env); - k_flush_executor->init(); + k_flush_executor = k_engine->memtable_flush_executor(); } void tear_down() { delete k_engine; k_engine = nullptr; - delete k_flush_executor; - k_flush_executor = nullptr; - system("rm -rf ./data_test"); + system("rm -rf ./flush_test"); remove_all_dir(std::string(getenv("DORIS_HOME")) + UNUSED_PREFIX); } @@ -96,17 +92,32 @@ class TestMemTableFlushExecutor : public ::testing::Test { } }; -TEST_F(TestMemTableFlushExecutor, get_queue_idx) { - for (auto store : k_engine->get_stores()) { - int32_t idx = k_flush_executor->get_queue_idx(store->path_hash()); - ASSERT_EQ(0, idx); - idx = k_flush_executor->get_queue_idx(store->path_hash()); - ASSERT_EQ(1, idx); - idx = k_flush_executor->get_queue_idx(store->path_hash()); - ASSERT_EQ(0, idx); - idx = k_flush_executor->get_queue_idx(store->path_hash()); - ASSERT_EQ(1, idx); - } +TEST_F(TestMemTableFlushExecutor, create_flush_handler) { + + std::vector data_dir = k_engine->get_stores(); + int64_t path_hash = data_dir[0]->path_hash(); + + FlushHandler* flush_handler; + k_flush_executor->create_flush_handler(path_hash, &flush_handler); + ASSERT_NE(nullptr, flush_handler); + + std::shared_ptr shared(flush_handler); + FlushResult res; + res.flush_status = OLAP_SUCCESS; + res.flush_time_ns = 100; + shared->on_flush_finished(res); + ASSERT_EQ(OLAP_SUCCESS, shared->last_flush_status()); + ASSERT_EQ(100, shared->get_stats().flush_time_ns); + ASSERT_EQ(1, shared->get_stats().flush_count); + + FlushResult res2; + res2.flush_status = OLAP_ERR_OTHER_ERROR; + shared->on_flush_finished(res2); + ASSERT_EQ(OLAP_ERR_OTHER_ERROR, shared->last_flush_status()); + ASSERT_EQ(100, shared->get_stats().flush_time_ns); + ASSERT_EQ(1, shared->get_stats().flush_count); + + ASSERT_EQ(OLAP_ERR_OTHER_ERROR, shared->wait()); } } // namespace doris diff --git a/be/test/runtime/CMakeLists.txt b/be/test/runtime/CMakeLists.txt index 0ca58bd08af863..ccfe08bc4b6af1 100644 --- a/be/test/runtime/CMakeLists.txt +++ b/be/test/runtime/CMakeLists.txt @@ -59,7 +59,6 @@ ADD_BE_TEST(user_function_cache_test) ADD_BE_TEST(kafka_consumer_pipe_test) ADD_BE_TEST(routine_load_task_executor_test) ADD_BE_TEST(small_file_mgr_test) -ADD_BE_TEST(memtable_flush_executor_test) ADD_BE_TEST(result_queue_mgr_test) ADD_BE_TEST(memory_scratch_sink_test) diff --git a/be/test/runtime/tablet_writer_mgr_test.cpp b/be/test/runtime/tablet_writer_mgr_test.cpp index 1fc8240540846f..d42ad849b2ad3e 100644 --- a/be/test/runtime/tablet_writer_mgr_test.cpp +++ b/be/test/runtime/tablet_writer_mgr_test.cpp @@ -27,12 +27,12 @@ #include "runtime/descriptors.h" #include "runtime/primitive_type.h" #include "runtime/mem_tracker.h" -// #include "runtime/memtable_flush_executor.h" #include "runtime/row_batch.h" #include "runtime/tuple_row.h" #include "runtime/descriptor_helper.h" #include "util/thrift_util.h" #include "olap/delta_writer.h" +#include "olap/storage_engine.h" namespace doris { @@ -43,7 +43,7 @@ OLAPStatus close_status; int64_t wait_lock_time_ns; // mock -DeltaWriter::DeltaWriter(WriteRequest* req, MemTableFlushExecutor* flush_executor) : _req(*req) { +DeltaWriter::DeltaWriter(WriteRequest* req, StorageEngine* storage_engine) : _req(*req) { } DeltaWriter::~DeltaWriter() { @@ -53,11 +53,11 @@ OLAPStatus DeltaWriter::init() { return OLAP_SUCCESS; } -OLAPStatus DeltaWriter::open(WriteRequest* req, MemTableFlushExecutor* flush_executor, DeltaWriter** writer) { +OLAPStatus DeltaWriter::open(WriteRequest* req, DeltaWriter** writer) { if (open_status != OLAP_SUCCESS) { return open_status; } - *writer = new DeltaWriter(req, flush_executor); + *writer = new DeltaWriter(req, nullptr); return open_status; } @@ -70,11 +70,11 @@ OLAPStatus DeltaWriter::write(Tuple* tuple) { return add_status; } -OLAPStatus DeltaWriter::flush() { +OLAPStatus DeltaWriter::close() { return OLAP_SUCCESS; } -OLAPStatus DeltaWriter::close(google::protobuf::RepeatedPtrField* tablet_vec) { +OLAPStatus DeltaWriter::close_wait(google::protobuf::RepeatedPtrField* tablet_vec) { return close_status; } diff --git a/run-ut.sh b/run-ut.sh index 42db0a81e5bf02..01731b28bc8a65 100755 --- a/run-ut.sh +++ b/run-ut.sh @@ -211,7 +211,6 @@ ${DORIS_TEST_BINARY_DIR}/runtime/snapshot_loader_test ${DORIS_TEST_BINARY_DIR}/runtime/user_function_cache_test ${DORIS_TEST_BINARY_DIR}/runtime/small_file_mgr_test ${DORIS_TEST_BINARY_DIR}/runtime/mem_pool_test -${DORIS_TEST_BINARY_DIR}/runtime/memtable_flush_executor_test ${DORIS_TEST_BINARY_DIR}/runtime/memory/chunk_allocator_test ${DORIS_TEST_BINARY_DIR}/runtime/memory/system_allocator_test # Running expr Unittest @@ -242,6 +241,7 @@ ${DORIS_TEST_BINARY_DIR}/olap/column_reader_test ${DORIS_TEST_BINARY_DIR}/olap/row_cursor_test ${DORIS_TEST_BINARY_DIR}/olap/skiplist_test ${DORIS_TEST_BINARY_DIR}/olap/serialize_test +${DORIS_TEST_BINARY_DIR}/olap/memtable_flush_executor_test # Running routine load test ${DORIS_TEST_BINARY_DIR}/olap/tablet_meta_manager_test From 0d9737976ec5c67ddea75c9315e9faef476d8d12 Mon Sep 17 00:00:00 2001 From: chenmingyu Date: Mon, 23 Sep 2019 21:58:25 +0800 Subject: [PATCH 5/8] fix by zc review 2 --- be/src/olap/delta_writer.cpp | 4 +--- be/src/olap/memtable_flush_executor.cpp | 8 ++----- be/src/olap/memtable_flush_executor.h | 7 +++--- be/test/olap/memtable_flush_executor_test.cpp | 23 +++++++++---------- 4 files changed, 17 insertions(+), 25 deletions(-) diff --git a/be/src/olap/delta_writer.cpp b/be/src/olap/delta_writer.cpp index 67261c95c7499f..0fea80a05d209c 100644 --- a/be/src/olap/delta_writer.cpp +++ b/be/src/olap/delta_writer.cpp @@ -141,9 +141,7 @@ OLAPStatus DeltaWriter::init() { _req.tuple_desc, _tablet->keys_type(), _rowset_writer.get()); // create flush handler - FlushHandler* flush_handler; - RETURN_NOT_OK(_storage_engine->memtable_flush_executor()->create_flush_handler(_tablet->data_dir()->path_hash(), &flush_handler)); - _flush_handler.reset(flush_handler); + RETURN_NOT_OK(_storage_engine->memtable_flush_executor()->create_flush_handler(_tablet->data_dir()->path_hash(), &_flush_handler)); _is_init = true; return OLAP_SUCCESS; diff --git a/be/src/olap/memtable_flush_executor.cpp b/be/src/olap/memtable_flush_executor.cpp index 90239d6fef38a1..85c06d3279d42b 100644 --- a/be/src/olap/memtable_flush_executor.cpp +++ b/be/src/olap/memtable_flush_executor.cpp @@ -50,10 +50,6 @@ OLAPStatus FlushHandler::wait() { return OLAP_SUCCESS; } -const FlushStatistic& FlushHandler::get_stats() { - return _stats; -} - void FlushHandler::on_flush_finished(const FlushResult& res) { if (res.flush_status != OLAP_SUCCESS) { _last_flush_status.store(res.flush_status); @@ -63,9 +59,9 @@ void FlushHandler::on_flush_finished(const FlushResult& res) { } } -OLAPStatus MemTableFlushExecutor::create_flush_handler(int64_t path_hash, FlushHandler** flush_handler) { +OLAPStatus MemTableFlushExecutor::create_flush_handler(int64_t path_hash, std::shared_ptr* flush_handler) { int32_t flush_queue_idx = _get_queue_idx(path_hash); - *flush_handler = new FlushHandler(flush_queue_idx, this); + flush_handler->reset(new FlushHandler(flush_queue_idx, this)); return OLAP_SUCCESS; } diff --git a/be/src/olap/memtable_flush_executor.h b/be/src/olap/memtable_flush_executor.h index cf7279cd26a28c..76cd16020aab11 100644 --- a/be/src/olap/memtable_flush_executor.h +++ b/be/src/olap/memtable_flush_executor.h @@ -84,7 +84,7 @@ class FlushHandler : public std::enable_shared_from_this { // wait for all submitted memtable finished. OLAPStatus wait(); // get flush operations' statistics - const FlushStatistic& get_stats(); + const FlushStatistic& get_stats() const { return _stats; } // called when a memtable is finished by executor. void on_flush_finished(const FlushResult& res); @@ -106,9 +106,8 @@ class FlushHandler : public std::enable_shared_from_this { // User SHOULD NOT call method of this class directly, use pattern should be: // // ... -// FlushHandler* flush_handler; +// std::shared_ptr flush_handler; // memTableFlushExecutor.create_flush_handler(path_hash, &flush_handler); -// std::shared_ptr shared_handler(flush_handler); // ... // flush_handler->submit(memtable) // ... @@ -122,7 +121,7 @@ class MemTableFlushExecutor { ~MemTableFlushExecutor(); // create a flush handler to access the flush executor - OLAPStatus create_flush_handler(int64_t path_hash, FlushHandler** flush_handler); + OLAPStatus create_flush_handler(int64_t path_hash, std::shared_ptr* flush_handler); private: diff --git a/be/test/olap/memtable_flush_executor_test.cpp b/be/test/olap/memtable_flush_executor_test.cpp index 4427e789a854ae..aec477a6eff40d 100644 --- a/be/test/olap/memtable_flush_executor_test.cpp +++ b/be/test/olap/memtable_flush_executor_test.cpp @@ -97,27 +97,26 @@ TEST_F(TestMemTableFlushExecutor, create_flush_handler) { std::vector data_dir = k_engine->get_stores(); int64_t path_hash = data_dir[0]->path_hash(); - FlushHandler* flush_handler; + std::flush_handler_ptr flush_handler; k_flush_executor->create_flush_handler(path_hash, &flush_handler); - ASSERT_NE(nullptr, flush_handler); + ASSERT_NE(nullptr, flush_handler.get()); - std::shared_ptr shared(flush_handler); FlushResult res; res.flush_status = OLAP_SUCCESS; res.flush_time_ns = 100; - shared->on_flush_finished(res); - ASSERT_EQ(OLAP_SUCCESS, shared->last_flush_status()); - ASSERT_EQ(100, shared->get_stats().flush_time_ns); - ASSERT_EQ(1, shared->get_stats().flush_count); + flush_handler->on_flush_finished(res); + ASSERT_EQ(OLAP_SUCCESS, flush_handler->last_flush_status()); + ASSERT_EQ(100, flush_handler->get_stats().flush_time_ns); + ASSERT_EQ(1, flush_handler->get_stats().flush_count); FlushResult res2; res2.flush_status = OLAP_ERR_OTHER_ERROR; - shared->on_flush_finished(res2); - ASSERT_EQ(OLAP_ERR_OTHER_ERROR, shared->last_flush_status()); - ASSERT_EQ(100, shared->get_stats().flush_time_ns); - ASSERT_EQ(1, shared->get_stats().flush_count); + flush_handler->on_flush_finished(res2); + ASSERT_EQ(OLAP_ERR_OTHER_ERROR, flush_handler->last_flush_status()); + ASSERT_EQ(100, flush_handler->get_stats().flush_time_ns); + ASSERT_EQ(1, flush_handler->get_stats().flush_count); - ASSERT_EQ(OLAP_ERR_OTHER_ERROR, shared->wait()); + ASSERT_EQ(OLAP_ERR_OTHER_ERROR, flush_handler->wait()); } } // namespace doris From 1235497a5a8e5e92f22f8f8dbd861f955f5f5f06 Mon Sep 17 00:00:00 2001 From: chenmingyu Date: Tue, 24 Sep 2019 17:31:45 +0800 Subject: [PATCH 6/8] add counter cond variable --- be/src/olap/memtable_flush_executor.cpp | 41 +++------ be/src/olap/memtable_flush_executor.h | 22 ++--- be/src/runtime/tablets_channel.cpp | 4 + be/src/util/counter_cond_variable.hpp | 73 +++++++++++++++ be/src/util/semaphore.hpp | 6 +- be/test/olap/memtable_flush_executor_test.cpp | 7 +- be/test/util/CMakeLists.txt | 1 + be/test/util/counter_cond_variable_test.cpp | 91 +++++++++++++++++++ .../transaction/GlobalTransactionMgr.java | 2 +- run-ut.sh | 1 + 10 files changed, 196 insertions(+), 52 deletions(-) create mode 100644 be/src/util/counter_cond_variable.hpp create mode 100644 be/test/util/counter_cond_variable_test.cpp diff --git a/be/src/olap/memtable_flush_executor.cpp b/be/src/olap/memtable_flush_executor.cpp index 85c06d3279d42b..fc2d66f3b21cef 100644 --- a/be/src/olap/memtable_flush_executor.cpp +++ b/be/src/olap/memtable_flush_executor.cpp @@ -29,25 +29,19 @@ OLAPStatus FlushHandler::submit(std::shared_ptr memtable) { MemTableFlushContext ctx; ctx.memtable = memtable; ctx.flush_handler = this->shared_from_this(); - _flush_futures.push(_flush_executor->_push_memtable(_flush_queue_idx, ctx)); + _counter_cond.inc(); + _flush_executor->_push_memtable(_flush_queue_idx, ctx); return OLAP_SUCCESS; } OLAPStatus FlushHandler::wait() { - if (_last_flush_status.load() != OLAP_SUCCESS) { - return _last_flush_status.load(); - } - - while(!_flush_futures.empty()) { - std::future& fu = _flush_futures.front(); - OLAPStatus st = fu.get(); - if (st != OLAP_SUCCESS) { - _last_flush_status.store(st); - return st; + // wait util encoutering error, or all submitted memtables are finished + while(_last_flush_status.load() == OLAP_SUCCESS) { + if (_counter_cond.check_wait()) { + break; } - _flush_futures.pop(); } - return OLAP_SUCCESS; + return _last_flush_status.load(); } void FlushHandler::on_flush_finished(const FlushResult& res) { @@ -57,6 +51,7 @@ void FlushHandler::on_flush_finished(const FlushResult& res) { _stats.flush_time_ns.fetch_add(res.flush_time_ns); _stats.flush_count.fetch_add(1); } + _counter_cond.dec(); } OLAPStatus MemTableFlushExecutor::create_flush_handler(int64_t path_hash, std::shared_ptr* flush_handler) { @@ -122,17 +117,8 @@ int32_t MemTableFlushExecutor::_get_queue_idx(size_t path_hash) { return cur_idx; } -std::future MemTableFlushExecutor::_push_memtable(int32_t queue_idx, MemTableFlushContext& ctx) { - ctx.flush_id = _id_generator.fetch_add(1); - std::promise promise; - std::future fu = promise.get_future(); - { - std::lock_guard l(_lock); - _flush_promises[ctx.flush_id] = std::move(promise); - } - +void MemTableFlushExecutor::_push_memtable(int32_t queue_idx, MemTableFlushContext& ctx) { _flush_queues[queue_idx]->blocking_put(ctx); - return fu; } void MemTableFlushExecutor::_flush_memtable(int32_t queue_idx) { @@ -144,7 +130,7 @@ void MemTableFlushExecutor::_flush_memtable(int32_t queue_idx) { } // if last flush of this tablet already failed, just skip - if (ctx.flush_handler->last_flush_status() != OLAP_SUCCESS) { + if (ctx.flush_handler->is_cancelled()) { continue; } @@ -154,13 +140,8 @@ void MemTableFlushExecutor::_flush_memtable(int32_t queue_idx) { timer.start(); res.flush_status = ctx.memtable->flush(); res.flush_time_ns = timer.elapsed_time(); + // callback ctx.flush_handler->on_flush_finished(res); - - { - std::lock_guard l(_lock); - _flush_promises[ctx.flush_id].set_value(res.flush_status); - _flush_promises.erase(ctx.flush_id); - } } } diff --git a/be/src/olap/memtable_flush_executor.h b/be/src/olap/memtable_flush_executor.h index 76cd16020aab11..4b90aedb32c3de 100644 --- a/be/src/olap/memtable_flush_executor.h +++ b/be/src/olap/memtable_flush_executor.h @@ -19,7 +19,6 @@ #include #include -#include #include #include #include @@ -28,6 +27,7 @@ #include "olap/olap_define.h" #include "util/blocking_queue.hpp" +#include "util/counter_cond_variable.hpp" #include "util/spinlock.h" #include "util/thread_pool.hpp" @@ -41,9 +41,6 @@ class MemTable; // The context for a memtable to be flushed. class FlushHandler; struct MemTableFlushContext { - // generated by memtable flush executor - // denote a certain memtable flush execution. - int64_t flush_id; // memtable to be flushed std::shared_ptr memtable; // flush handler from a delta writer. @@ -76,6 +73,7 @@ class FlushHandler : public std::enable_shared_from_this { FlushHandler(int32_t flush_queue_idx, MemTableFlushExecutor* flush_executor): _flush_queue_idx(flush_queue_idx), _last_flush_status(OLAP_SUCCESS), + _counter_cond(0), _flush_executor(flush_executor) { } @@ -88,14 +86,15 @@ class FlushHandler : public std::enable_shared_from_this { // called when a memtable is finished by executor. void on_flush_finished(const FlushResult& res); - OLAPStatus last_flush_status() { return _last_flush_status.load(); } + bool is_cancelled() { return _last_flush_status.load() != OLAP_SUCCESS; } private: // flush queue idx in memtable flush executor int32_t _flush_queue_idx; // the flush status of last memtable std::atomic _last_flush_status; - // save all futures of memtable flush execution. - std::queue> _flush_futures; + // used to wait/notify the memtable flush execution + CounterCondVariable _counter_cond; + FlushStatistic _stats; MemTableFlushExecutor* _flush_executor; }; @@ -124,15 +123,14 @@ class MemTableFlushExecutor { OLAPStatus create_flush_handler(int64_t path_hash, std::shared_ptr* flush_handler); private: - // given the path hash, return the next idx of flush queue. // eg. // path A is mapped to idx 0 and 1, so each time get_queue_idx(A) is called, // 0 and 1 will returned alternately. int32_t _get_queue_idx(size_t path_hash); - // push the memtable to specified flush queue, and return a future - std::future _push_memtable(int32_t queue_idx, MemTableFlushContext& ctx); + // push the memtable to specified flush queue + void _push_memtable(int32_t queue_idx, MemTableFlushContext& ctx); void _flush_memtable(int32_t queue_idx); @@ -144,14 +142,10 @@ class MemTableFlushExecutor { ThreadPool* _flush_pool; // the size of this vector should equals to _num_threads std::vector*> _flush_queues; - - std::atomic_int64_t _id_generator; - // lock to protect path_map SpinLock _lock; // path hash -> queue idx of _flush_queues; std::unordered_map _path_map; - std::unordered_map> _flush_promises; }; } // end namespace diff --git a/be/src/runtime/tablets_channel.cpp b/be/src/runtime/tablets_channel.cpp index 5a299fcf4ab278..b52f1aa028faa8 100644 --- a/be/src/runtime/tablets_channel.cpp +++ b/be/src/runtime/tablets_channel.cpp @@ -27,6 +27,10 @@ namespace doris { TabletsChannel::TabletsChannel(const TabletsChannelKey& key): _key(key), _closed_senders(64) { + // _last_updated_time should be set before being inserted to + // _tablet_channels in tablet_channel_mgr, or it may be erased + // immediately by gc thread. + _last_updated_time = time(nullptr); } TabletsChannel::~TabletsChannel() { diff --git a/be/src/util/counter_cond_variable.hpp b/be/src/util/counter_cond_variable.hpp new file mode 100644 index 00000000000000..18c9692e2da55b --- /dev/null +++ b/be/src/util/counter_cond_variable.hpp @@ -0,0 +1,73 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include + +namespace doris { + +// used for submitter/worker/waiter pattern +// submitter: +// one or more submitters submit tasks and call inc_count() +// worker: +// one or more workers do the task and call dec_count() after finishing the task +// waiter: +// one or more waiter call xxx_wait() to wait until all or at least one tasks are finished. +class CounterCondVariable { + public: + explicit CounterCondVariable(int init = 0) : _count(init) { + } + + // increase the counter + void inc(int inc = 1) { + std::unique_lock lock(_lock); + _count += inc; + } + + // decrease the counter, and notify all waiters + void dec(int dec = 1) { + std::unique_lock lock(_lock); + _count -= dec; + _cv.notify_all(); + } + + // wait until count down to zero + void block_wait() { + std::unique_lock lock(_lock); + _cv.wait(lock, [=] { return _count <= 0; }); + } + + // wait if count larger than 0 + // and after being notified, return true if count down zo zero, + // or return false other wise. + bool check_wait() { + std::unique_lock lock(_lock); + if (_count > 0) { + _cv.wait(lock); + } + return _count <= 0; + } + + private: + std::mutex _lock; + std::condition_variable _cv; + int _count; +}; + +} // end namespace diff --git a/be/src/util/semaphore.hpp b/be/src/util/semaphore.hpp index 398c5ae3ed6cab..dac287d604f74f 100644 --- a/be/src/util/semaphore.hpp +++ b/be/src/util/semaphore.hpp @@ -31,13 +31,13 @@ class Semaphore { void signal() { std::unique_lock lock(_mutex); - ++count_; - cv_.notify_one(); + ++_count; + _cv.notify_one(); } void wait() { std::unique_lock lock(_mutex); - cv_.wait(lock, [=] { return _count > 0; }); + _cv.wait(lock, [=] { return _count > 0; }); --_count; } diff --git a/be/test/olap/memtable_flush_executor_test.cpp b/be/test/olap/memtable_flush_executor_test.cpp index aec477a6eff40d..cfbcab538c6041 100644 --- a/be/test/olap/memtable_flush_executor_test.cpp +++ b/be/test/olap/memtable_flush_executor_test.cpp @@ -93,11 +93,10 @@ class TestMemTableFlushExecutor : public ::testing::Test { }; TEST_F(TestMemTableFlushExecutor, create_flush_handler) { - std::vector data_dir = k_engine->get_stores(); int64_t path_hash = data_dir[0]->path_hash(); - std::flush_handler_ptr flush_handler; + std::shared_ptr flush_handler; k_flush_executor->create_flush_handler(path_hash, &flush_handler); ASSERT_NE(nullptr, flush_handler.get()); @@ -105,14 +104,14 @@ TEST_F(TestMemTableFlushExecutor, create_flush_handler) { res.flush_status = OLAP_SUCCESS; res.flush_time_ns = 100; flush_handler->on_flush_finished(res); - ASSERT_EQ(OLAP_SUCCESS, flush_handler->last_flush_status()); + ASSERT_FALSE(flush_handler->is_cancelled()); ASSERT_EQ(100, flush_handler->get_stats().flush_time_ns); ASSERT_EQ(1, flush_handler->get_stats().flush_count); FlushResult res2; res2.flush_status = OLAP_ERR_OTHER_ERROR; flush_handler->on_flush_finished(res2); - ASSERT_EQ(OLAP_ERR_OTHER_ERROR, flush_handler->last_flush_status()); + ASSERT_TRUE(flush_handler->is_cancelled()); ASSERT_EQ(100, flush_handler->get_stats().flush_time_ns); ASSERT_EQ(1, flush_handler->get_stats().flush_count); diff --git a/be/test/util/CMakeLists.txt b/be/test/util/CMakeLists.txt index 1955a004524e34..ff910ddfa23abb 100644 --- a/be/test/util/CMakeLists.txt +++ b/be/test/util/CMakeLists.txt @@ -47,3 +47,4 @@ ADD_BE_TEST(tdigest_test) ADD_BE_TEST(block_compression_test) ADD_BE_TEST(arrow/arrow_row_block_test) ADD_BE_TEST(arrow/arrow_row_batch_test) +ADD_BE_TEST(counter_cond_variable_test) diff --git a/be/test/util/counter_cond_variable_test.cpp b/be/test/util/counter_cond_variable_test.cpp new file mode 100644 index 00000000000000..79aabc61554721 --- /dev/null +++ b/be/test/util/counter_cond_variable_test.cpp @@ -0,0 +1,91 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "util/counter_cond_variable.hpp" + +#include +#include + +#include "common/logging.h" + +namespace doris { + +CounterCondVariable g_cond; +std::mutex g_io_mu; + +class CounterCondVariableTest : public testing::Test { +public: + CounterCondVariableTest() { } + virtual ~CounterCondVariableTest() { } +}; + +void submitter() { + for (int i = 0; i < 10; ++i) { + g_cond.inc(); + } +} + +void worker() { + for (int i = 0; i < 10; ++i) { + { + std::unique_lock lock(g_io_mu); + std::cout << "worker " << i << std::endl; + } + sleep(1); + g_cond.dec(); + } +} + +void waiter() { + while (!g_cond.check_wait()) { + { + std::unique_lock lock(g_io_mu); + std::cout << "wait " << std::endl; + } + sleep(1); + } + std::cout << "wait finished" << std::endl; +} + + +TEST_F(CounterCondVariableTest, test) { + g_cond.block_wait(); + g_cond.inc(10); + g_cond.dec(10); + g_cond.block_wait(); + ASSERT_TRUE(g_cond.check_wait()); + + std::thread submit(submitter); + std::thread wait1(waiter); + std::thread wait2(waiter); + std::thread work1(worker); + std::thread work2(worker); + + submit.join(); + wait1.join(); + wait2.join(); + work1.join(); + work2.join(); + ASSERT_TRUE(g_cond.check_wait()); +} + +} + +int main(int argc, char* argv[]) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java b/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java index c83aa89a74468c..a0331d432d4722 100644 --- a/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java +++ b/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java @@ -395,7 +395,7 @@ public void commitTransaction(long dbId, long transactionId, List Date: Tue, 24 Sep 2019 20:49:11 +0800 Subject: [PATCH 7/8] fix by review --- be/src/olap/memtable_flush_executor.cpp | 11 ++- be/src/util/counter_cond_variable.hpp | 83 +++++++++++++-------- be/test/util/counter_cond_variable_test.cpp | 15 ++-- 3 files changed, 61 insertions(+), 48 deletions(-) diff --git a/be/src/olap/memtable_flush_executor.cpp b/be/src/olap/memtable_flush_executor.cpp index fc2d66f3b21cef..50e320b990fd7c 100644 --- a/be/src/olap/memtable_flush_executor.cpp +++ b/be/src/olap/memtable_flush_executor.cpp @@ -36,22 +36,21 @@ OLAPStatus FlushHandler::submit(std::shared_ptr memtable) { OLAPStatus FlushHandler::wait() { // wait util encoutering error, or all submitted memtables are finished - while(_last_flush_status.load() == OLAP_SUCCESS) { - if (_counter_cond.check_wait()) { - break; - } - } + RETURN_NOT_OK(_last_flush_status.load()); + _counter_cond.block_wait(); return _last_flush_status.load(); } void FlushHandler::on_flush_finished(const FlushResult& res) { if (res.flush_status != OLAP_SUCCESS) { _last_flush_status.store(res.flush_status); + // if one failed, all other memtables no need to flush + _counter_cond.dec_to_zero(); } else { _stats.flush_time_ns.fetch_add(res.flush_time_ns); _stats.flush_count.fetch_add(1); + _counter_cond.dec(); } - _counter_cond.dec(); } OLAPStatus MemTableFlushExecutor::create_flush_handler(int64_t path_hash, std::shared_ptr* flush_handler) { diff --git a/be/src/util/counter_cond_variable.hpp b/be/src/util/counter_cond_variable.hpp index 18c9692e2da55b..9048037826207b 100644 --- a/be/src/util/counter_cond_variable.hpp +++ b/be/src/util/counter_cond_variable.hpp @@ -29,45 +29,64 @@ namespace doris { // one or more workers do the task and call dec_count() after finishing the task // waiter: // one or more waiter call xxx_wait() to wait until all or at least one tasks are finished. +// Use pattern: +// thread1(submitter): +// CounterCondVariable cond(0); +// ... submit task ... +// cond.inc(); +// ... submit task ... +// cond.inr(); +// +// thread2(worker): +// ... do work... +// cond.dec(); +// ... do work... +// cond.dec(); +// or +// ... failed ... +// cond.dec_to_zero(); +// +// thread3(waiter): +// cond.block_wait(); + class CounterCondVariable { - public: - explicit CounterCondVariable(int init = 0) : _count(init) { - } +public: + explicit CounterCondVariable(int init = 0) : _count(init) { + } - // increase the counter - void inc(int inc = 1) { - std::unique_lock lock(_lock); - _count += inc; - } + // increase the counter + void inc(int inc = 1) { + std::unique_lock lock(_lock); + _count += inc; + } - // decrease the counter, and notify all waiters - void dec(int dec = 1) { - std::unique_lock lock(_lock); - _count -= dec; - _cv.notify_all(); - } + // decrease the counter, and notify all waiters + void dec(int dec = 1) { + std::unique_lock lock(_lock); + _count -= dec; + _cv.notify_all(); + } - // wait until count down to zero - void block_wait() { - std::unique_lock lock(_lock); - _cv.wait(lock, [=] { return _count <= 0; }); - } + // decrease the counter to zero + void dec_to_zero() { + std::unique_lock lock(_lock); + _count = 0; + _cv.notify_all(); + } - // wait if count larger than 0 - // and after being notified, return true if count down zo zero, - // or return false other wise. - bool check_wait() { - std::unique_lock lock(_lock); - if (_count > 0) { - _cv.wait(lock); - } - return _count <= 0; + // wait until count down to zero + void block_wait() { + std::unique_lock lock(_lock); + if (_count <= 0) { + return; } + _cv.wait(lock, [this] { return _count <= 0; }); + } - private: - std::mutex _lock; - std::condition_variable _cv; - int _count; +private: + std::mutex _lock; + std::condition_variable _cv; + int _count; }; } // end namespace diff --git a/be/test/util/counter_cond_variable_test.cpp b/be/test/util/counter_cond_variable_test.cpp index 79aabc61554721..b64110af5424bf 100644 --- a/be/test/util/counter_cond_variable_test.cpp +++ b/be/test/util/counter_cond_variable_test.cpp @@ -51,23 +51,15 @@ void worker() { } void waiter() { - while (!g_cond.check_wait()) { - { - std::unique_lock lock(g_io_mu); - std::cout << "wait " << std::endl; - } - sleep(1); - } + g_cond.block_wait(); std::cout << "wait finished" << std::endl; } - TEST_F(CounterCondVariableTest, test) { g_cond.block_wait(); g_cond.inc(10); g_cond.dec(10); g_cond.block_wait(); - ASSERT_TRUE(g_cond.check_wait()); std::thread submit(submitter); std::thread wait1(waiter); @@ -80,7 +72,10 @@ TEST_F(CounterCondVariableTest, test) { wait2.join(); work1.join(); work2.join(); - ASSERT_TRUE(g_cond.check_wait()); + + g_cond.inc(10); + g_cond.dec_to_zero(); + g_cond.block_wait(); } } From 91b3ac75c6e62c49c84fb8b98f5ca671a55cc043 Mon Sep 17 00:00:00 2001 From: chenmingyu Date: Wed, 25 Sep 2019 09:31:51 +0800 Subject: [PATCH 8/8] add on_flush_cancelled callback --- be/src/olap/memtable.h | 3 --- be/src/olap/memtable_flush_executor.cpp | 1 + be/src/olap/memtable_flush_executor.h | 5 +++++ 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/be/src/olap/memtable.h b/be/src/olap/memtable.h index 5fb428e03f471f..80be744cf7708d 100644 --- a/be/src/olap/memtable.h +++ b/be/src/olap/memtable.h @@ -18,9 +18,6 @@ #ifndef DORIS_BE_SRC_OLAP_MEMTABLE_H #define DORIS_BE_SRC_OLAP_MEMTABLE_H -#include -#include - #include "olap/schema.h" #include "olap/skiplist.h" #include "runtime/tuple.h" diff --git a/be/src/olap/memtable_flush_executor.cpp b/be/src/olap/memtable_flush_executor.cpp index 50e320b990fd7c..a2a24485a78251 100644 --- a/be/src/olap/memtable_flush_executor.cpp +++ b/be/src/olap/memtable_flush_executor.cpp @@ -130,6 +130,7 @@ void MemTableFlushExecutor::_flush_memtable(int32_t queue_idx) { // if last flush of this tablet already failed, just skip if (ctx.flush_handler->is_cancelled()) { + ctx.flush_handler->on_flush_cancelled(); continue; } diff --git a/be/src/olap/memtable_flush_executor.h b/be/src/olap/memtable_flush_executor.h index 4b90aedb32c3de..dc5b5d24c070d2 100644 --- a/be/src/olap/memtable_flush_executor.h +++ b/be/src/olap/memtable_flush_executor.h @@ -85,6 +85,11 @@ class FlushHandler : public std::enable_shared_from_this { const FlushStatistic& get_stats() const { return _stats; } // called when a memtable is finished by executor. void on_flush_finished(const FlushResult& res); + // called when a flush memtable execution is cancelled + void on_flush_cancelled() { + // for now, if one memtable cancelled, no more memtables will be flushed, so dec to zero. + _counter_cond.dec_to_zero(); + } bool is_cancelled() { return _last_flush_status.load() != OLAP_SUCCESS; } private: