From e789b0fd98de20fc7cd66989e5e7fdba10591f2d Mon Sep 17 00:00:00 2001 From: Sun Chenyang Date: Wed, 30 Oct 2024 10:39:17 +0800 Subject: [PATCH 1/2] [Refactor] (inverted index) Refactor Inverted index file writer (#41625) ## Proposed changes 1. After the normal segment is flushed, the `close_inverted_index` is directly called to write the final composite file. 2. During compaction, in the first step, the `segment writer `writes the `bkd index` while writing normal data. In the second step, the` index compaction` writes the `string index`. In the third step, `close_inverted_index` is uniformly called for all indexes to write the final files. 3. The rowset writer uses `InvertedIndexFileCollection` to store all inverted index file writers, ensuring their lifecycle exists throughout the entire writing or compaction process. 4. When the rowset writer generates the final rowset through `build(rowset)`, it can retrieve the index file sizes from the `InvertedIndexFileCollection` and record them in the rowset meta. --- be/src/cloud/cloud_rowset_writer.cpp | 15 +- be/src/common/config.cpp | 2 +- be/src/olap/compaction.cpp | 158 +-- be/src/olap/compaction.h | 1 + be/src/olap/rowset/beta_rowset_writer.cpp | 109 +- be/src/olap/rowset/beta_rowset_writer.h | 87 +- be/src/olap/rowset/beta_rowset_writer_v2.cpp | 2 +- be/src/olap/rowset/beta_rowset_writer_v2.h | 3 +- be/src/olap/rowset/rowset_meta.cpp | 10 +- be/src/olap/rowset/rowset_meta.h | 5 +- be/src/olap/rowset/rowset_writer.h | 19 + be/src/olap/rowset/segment_creator.cpp | 62 +- be/src/olap/rowset/segment_creator.h | 17 +- .../segment_v2/inverted_index_file_writer.cpp | 14 +- .../segment_v2/inverted_index_file_writer.h | 39 +- .../olap/rowset/segment_v2/segment_writer.cpp | 28 +- .../olap/rowset/segment_v2/segment_writer.h | 24 +- .../segment_v2/vertical_segment_writer.cpp | 30 +- .../segment_v2/vertical_segment_writer.h | 25 +- .../rowset/vertical_beta_rowset_writer.cpp | 36 +- be/src/olap/tablet_manager.cpp | 2 + be/src/olap/tablet_meta.h | 5 + be/src/olap/task/index_builder.cpp | 24 +- .../olap/delete_bitmap_calculator_test.cpp | 3 +- .../compaction/index_compaction_test.cpp | 443 ++++++++ .../index_compaction_write_index_test.cpp} | 8 +- .../segment_v2/inverted_index/data/data1.csv | 1000 +++++++++++++++++ .../segment_v2/inverted_index/data/data2.csv | 1000 +++++++++++++++++ .../test_index_compaction_p0.groovy | 151 +++ .../test_index_compaction_p1.groovy | 151 +++ 30 files changed, 3133 insertions(+), 340 deletions(-) create mode 100644 be/test/olap/rowset/segment_v2/inverted_index/compaction/index_compaction_test.cpp rename be/test/olap/rowset/segment_v2/inverted_index/{index_compaction_test.cpp => compaction/index_compaction_write_index_test.cpp} (96%) create mode 100644 be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv create mode 100644 be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv create mode 100644 regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_p0.groovy create mode 100644 regression-test/suites/inverted_index_p1/index_compaction/test_index_compaction_p1.groovy diff --git a/be/src/cloud/cloud_rowset_writer.cpp b/be/src/cloud/cloud_rowset_writer.cpp index 5f878f59d5c64d..754edd7762f9a9 100644 --- a/be/src/cloud/cloud_rowset_writer.cpp +++ b/be/src/cloud/cloud_rowset_writer.cpp @@ -115,13 +115,14 @@ Status CloudRowsetWriter::build(RowsetSharedPtr& rowset) { } else { _rowset_meta->add_segments_file_size(seg_file_size.value()); } - - if (auto idx_files_info = _idx_files_info.get_inverted_files_info(_segment_start_id); - !idx_files_info.has_value()) [[unlikely]] { - LOG(ERROR) << "expected inverted index files info, but none presents: " - << idx_files_info.error(); - } else { - _rowset_meta->add_inverted_index_files_info(idx_files_info.value()); + if (rowset_schema->has_inverted_index()) { + if (auto idx_files_info = _idx_files.inverted_index_file_info(_segment_start_id); + !idx_files_info.has_value()) [[unlikely]] { + LOG(ERROR) << "expected inverted index files info, but none presents: " + << idx_files_info.error(); + } else { + _rowset_meta->add_inverted_index_files_info(idx_files_info.value()); + } } RETURN_NOT_OK_STATUS_WITH_WARN(RowsetFactory::create_rowset(rowset_schema, _context.tablet_path, diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 7de013bb7a52ca..800eab8fd9ef40 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1070,7 +1070,7 @@ DEFINE_Int32(inverted_index_read_buffer_size, "4096"); // tree depth for bkd index DEFINE_Int32(max_depth_in_bkd_tree, "32"); // index compaction -DEFINE_mBool(inverted_index_compaction_enable, "false"); +DEFINE_mBool(inverted_index_compaction_enable, "true"); // Only for debug, do not use in production DEFINE_mBool(debug_inverted_index_compaction, "false"); // index by RAM directory diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index 0fd9b57faf8b93..8e5f68872239f4 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -186,6 +186,7 @@ Status Compaction::merge_input_rowsets() { Status res; { SCOPED_TIMER(_merge_rowsets_latency_timer); + // 1. Merge segment files and write bkd inverted index if (_is_vertical) { res = Merger::vertical_merge_rowsets(_tablet, compaction_type(), *_cur_tablet_schema, input_rs_readers, _output_rs_writer.get(), @@ -194,17 +195,19 @@ Status Compaction::merge_input_rowsets() { res = Merger::vmerge_rowsets(_tablet, compaction_type(), *_cur_tablet_schema, input_rs_readers, _output_rs_writer.get(), &_stats); } - } - - _tablet->last_compaction_status = res; - if (!res.ok()) { - return res; + _tablet->last_compaction_status = res; + if (!res.ok()) { + return res; + } + // 2. Merge the remaining inverted index files of the string type + RETURN_IF_ERROR(do_inverted_index_compaction()); } COUNTER_UPDATE(_merged_rows_counter, _stats.merged_rows); COUNTER_UPDATE(_filtered_rows_counter, _stats.filtered_rows); + // 3. In the `build`, `_close_file_writers` is called to close the inverted index file writer and write the final compound index file. RETURN_NOT_OK_STATUS_WITH_WARN(_output_rs_writer->build(_output_rowset), fmt::format("rowset writer build failed. output_version: {}", _output_version.to_string())); @@ -450,8 +453,6 @@ Status CompactionMixin::execute_compact_impl(int64_t permits) { RETURN_IF_ERROR(merge_input_rowsets()); - RETURN_IF_ERROR(do_inverted_index_compaction()); - RETURN_IF_ERROR(modify_rowsets()); auto* cumu_policy = tablet()->cumulative_compaction_policy(); @@ -607,58 +608,9 @@ Status Compaction::do_inverted_index_compaction() { // dest index files // format: rowsetId_segmentId - std::vector> inverted_index_file_writers( - dest_segment_num); - - // Some columns have already been indexed - // key: seg_id, value: inverted index file size - std::unordered_map compacted_idx_file_size; - for (int seg_id = 0; seg_id < dest_segment_num; ++seg_id) { - std::string index_path_prefix { - InvertedIndexDescriptor::get_index_file_path_prefix(ctx.segment_path(seg_id))}; - auto inverted_index_file_reader = std::make_unique( - ctx.fs(), index_path_prefix, - _cur_tablet_schema->get_inverted_index_storage_format()); - bool open_idx_file_cache = false; - auto st = inverted_index_file_reader->init(config::inverted_index_read_buffer_size, - open_idx_file_cache); - if (st.ok()) { - auto index_not_need_to_compact = - DORIS_TRY(inverted_index_file_reader->get_all_directories()); - // V1: each index is a separate file - // V2: all indexes are in a single file - if (_cur_tablet_schema->get_inverted_index_storage_format() != - doris::InvertedIndexStorageFormatPB::V1) { - int64_t fsize = 0; - st = ctx.fs()->file_size( - InvertedIndexDescriptor::get_index_file_path_v2(index_path_prefix), &fsize); - if (!st.ok()) { - LOG(ERROR) << "file size error in index compaction, error:" << st.msg(); - return st; - } - compacted_idx_file_size[seg_id] = fsize; - } - auto inverted_index_file_writer = std::make_unique( - ctx.fs(), index_path_prefix, ctx.rowset_id.to_string(), seg_id, - _cur_tablet_schema->get_inverted_index_storage_format()); - RETURN_IF_ERROR(inverted_index_file_writer->initialize(index_not_need_to_compact)); - inverted_index_file_writers[seg_id] = std::move(inverted_index_file_writer); - } else if (st.is()) { - auto inverted_index_file_writer = std::make_unique( - ctx.fs(), index_path_prefix, ctx.rowset_id.to_string(), seg_id, - _cur_tablet_schema->get_inverted_index_storage_format()); - inverted_index_file_writers[seg_id] = std::move(inverted_index_file_writer); - // no index file - compacted_idx_file_size[seg_id] = 0; - } else { - LOG(ERROR) << "inverted_index_file_reader init failed in index compaction, error:" - << st; - return st; - } - } - for (const auto& writer : inverted_index_file_writers) { - writer->set_file_writer_opts(ctx.get_file_writer_options()); - } + auto& inverted_index_file_writers = dynamic_cast(_output_rs_writer.get()) + ->inverted_index_file_writers(); + DCHECK_EQ(inverted_index_file_writers.size(), dest_segment_num); // use tmp file dir to store index files auto tmp_file_dir = ExecEnv::GetInstance()->get_tmp_file_dirs()->get_tmp_file_dir(); @@ -684,29 +636,6 @@ Status Compaction::do_inverted_index_compaction() { auto col = _cur_tablet_schema->column_by_uid(column_uniq_id); const auto* index_meta = _cur_tablet_schema->get_inverted_index(col); - // if index properties are different, index compaction maybe needs to be skipped. - bool is_continue = false; - std::optional> first_properties; - for (const auto& rowset : _input_rowsets) { - const auto* tablet_index = rowset->tablet_schema()->get_inverted_index(col); - const auto& properties = tablet_index->properties(); - if (!first_properties.has_value()) { - first_properties = properties; - } else { - if (properties != first_properties.value()) { - error_handler(index_meta->index_id(), column_uniq_id); - status = Status::Error( - "if index properties are different, index compaction needs to be " - "skipped."); - is_continue = true; - break; - } - } - } - if (is_continue) { - continue; - } - std::vector dest_index_dirs(dest_segment_num); try { std::vector> src_idx_dirs(src_segment_num); @@ -731,40 +660,12 @@ Status Compaction::do_inverted_index_compaction() { } } - std::vector all_inverted_index_file_info(dest_segment_num); - uint64_t inverted_index_file_size = 0; - for (int seg_id = 0; seg_id < dest_segment_num; ++seg_id) { - auto inverted_index_file_writer = inverted_index_file_writers[seg_id].get(); - if (Status st = inverted_index_file_writer->close(); !st.ok()) { - status = Status::Error(st.msg()); - } else { - inverted_index_file_size += inverted_index_file_writer->get_index_file_total_size(); - inverted_index_file_size -= compacted_idx_file_size[seg_id]; - } - all_inverted_index_file_info[seg_id] = inverted_index_file_writer->get_index_file_info(); - } // check index compaction status. If status is not ok, we should return error and end this compaction round. if (!status.ok()) { return status; } - - // index compaction should update total disk size and index disk size - _output_rowset->rowset_meta()->set_data_disk_size(_output_rowset->data_disk_size() + - inverted_index_file_size); - _output_rowset->rowset_meta()->set_total_disk_size(_output_rowset->data_disk_size() + - inverted_index_file_size); - _output_rowset->rowset_meta()->set_index_disk_size(_output_rowset->index_disk_size() + - inverted_index_file_size); - - _output_rowset->rowset_meta()->update_inverted_index_files_info(all_inverted_index_file_info); - COUNTER_UPDATE(_output_rowset_data_size_counter, _output_rowset->data_disk_size()); - LOG(INFO) << "succeed to do index compaction" - << ". tablet=" << _tablet->tablet_id() << ", input row number=" << _input_row_num - << ", output row number=" << _output_rowset->num_rows() - << ", input_rowset_size=" << _input_rowsets_size - << ", output_rowset_size=" << _output_rowset->data_disk_size() - << ", inverted index file size=" << inverted_index_file_size + << ". tablet=" << _tablet->tablet_id() << ". elapsed time=" << inverted_watch.get_elapse_second() << "s."; return Status::OK(); @@ -789,6 +690,31 @@ void Compaction::construct_index_compaction_columns(RowsetWriterContext& ctx) { if (!field_is_slice_type(_cur_tablet_schema->column_by_uid(col_unique_id).type())) { continue; } + + // if index properties are different, index compaction maybe needs to be skipped. + bool is_continue = false; + std::optional> first_properties; + for (const auto& rowset : _input_rowsets) { + const auto* tablet_index = + rowset->tablet_schema()->get_inverted_index(col_unique_id, ""); + // no inverted index or index id is different from current index id + if (tablet_index == nullptr || tablet_index->index_id() != index.index_id()) { + is_continue = true; + break; + } + const auto& properties = tablet_index->properties(); + if (!first_properties.has_value()) { + first_properties = properties; + } else { + if (properties != first_properties.value()) { + is_continue = true; + break; + } + } + } + if (is_continue) { + continue; + } auto has_inverted_index = [&](const RowsetSharedPtr& src_rs) { auto* rowset = static_cast(src_rs.get()); if (rowset->is_skip_index_compaction(col_unique_id)) { @@ -881,9 +807,7 @@ Status CompactionMixin::construct_output_rowset_writer(RowsetWriterContext& ctx) if (config::inverted_index_compaction_enable && (((_tablet->keys_type() == KeysType::UNIQUE_KEYS && _tablet->enable_unique_key_merge_on_write()) || - _tablet->keys_type() == KeysType::DUP_KEYS)) && - _cur_tablet_schema->get_inverted_index_storage_format() == - InvertedIndexStorageFormatPB::V1) { + _tablet->keys_type() == KeysType::DUP_KEYS))) { construct_index_compaction_columns(ctx); } ctx.version = _output_version; @@ -1149,8 +1073,6 @@ Status CloudCompactionMixin::execute_compact_impl(int64_t permits) { RETURN_IF_ERROR(merge_input_rowsets()); - RETURN_IF_ERROR(do_inverted_index_compaction()); - RETURN_IF_ERROR(_engine.meta_mgr().commit_rowset(*_output_rowset->rowset_meta().get())); // 4. modify rowsets in memory @@ -1177,9 +1099,7 @@ Status CloudCompactionMixin::construct_output_rowset_writer(RowsetWriterContext& if (config::inverted_index_compaction_enable && (((_tablet->keys_type() == KeysType::UNIQUE_KEYS && _tablet->enable_unique_key_merge_on_write()) || - _tablet->keys_type() == KeysType::DUP_KEYS)) && - _cur_tablet_schema->get_inverted_index_storage_format() == - InvertedIndexStorageFormatPB::V1) { + _tablet->keys_type() == KeysType::DUP_KEYS))) { construct_index_compaction_columns(ctx); } diff --git a/be/src/olap/compaction.h b/be/src/olap/compaction.h index f53b9004a796b5..b76573b445f3fd 100644 --- a/be/src/olap/compaction.h +++ b/be/src/olap/compaction.h @@ -67,6 +67,7 @@ class Compaction { protected: Status merge_input_rowsets(); + // merge inverted index files Status do_inverted_index_compaction(); void construct_index_compaction_columns(RowsetWriterContext& ctx); diff --git a/be/src/olap/rowset/beta_rowset_writer.cpp b/be/src/olap/rowset/beta_rowset_writer.cpp index ed659b601e83b1..a19989af1e7f85 100644 --- a/be/src/olap/rowset/beta_rowset_writer.cpp +++ b/be/src/olap/rowset/beta_rowset_writer.cpp @@ -189,13 +189,67 @@ Result> SegmentFileCollection::segments_file_size(int seg_id return ResultError(st); } +InvertedIndexFileCollection::~InvertedIndexFileCollection() = default; + +Status InvertedIndexFileCollection::add(int seg_id, InvertedIndexFileWriterPtr&& index_writer) { + std::lock_guard lock(_lock); + if (_inverted_index_file_writers.find(seg_id) != _inverted_index_file_writers.end()) + [[unlikely]] { + DCHECK(false); + return Status::InternalError("The seg_id already exists, seg_id is {}", seg_id); + } + _inverted_index_file_writers.emplace(seg_id, std::move(index_writer)); + return Status::OK(); +} + +Status InvertedIndexFileCollection::close() { + std::lock_guard lock(_lock); + for (auto&& [id, writer] : _inverted_index_file_writers) { + RETURN_IF_ERROR(writer->close()); + _total_size += writer->get_index_file_total_size(); + } + + return Status::OK(); +} + +Result> +InvertedIndexFileCollection::inverted_index_file_info(int seg_id_offset) { + std::lock_guard lock(_lock); + + Status st; + std::vector idx_file_info(_inverted_index_file_writers.size()); + bool succ = std::all_of( + _inverted_index_file_writers.begin(), _inverted_index_file_writers.end(), + [&](auto&& it) { + auto&& [seg_id, writer] = it; + + int idx = seg_id - seg_id_offset; + if (idx >= idx_file_info.size()) [[unlikely]] { + auto err_msg = + fmt::format("invalid seg_id={} num_file_writers={} seg_id_offset={}", + seg_id, idx_file_info.size(), seg_id_offset); + DCHECK(false) << err_msg; + st = Status::InternalError(err_msg); + return false; + } + idx_file_info[idx] = _inverted_index_file_writers[seg_id]->get_index_file_info(); + return true; + }); + + if (succ) { + return idx_file_info; + } + + return ResultError(st); +} + BaseBetaRowsetWriter::BaseBetaRowsetWriter() : _num_segment(0), _segment_start_id(0), _num_rows_written(0), _total_data_size(0), _total_index_size(0), - _segment_creator(_context, _seg_files, _idx_files_info) {} + _segment_creator(_context, _seg_files, _idx_files) {} BetaRowsetWriter::BetaRowsetWriter(StorageEngine& engine) : _engine(engine), _segcompaction_worker(std::make_shared(this)) {} @@ -721,7 +775,6 @@ Status BetaRowsetWriter::_close_file_writers() { Status BetaRowsetWriter::build(RowsetSharedPtr& rowset) { RETURN_IF_ERROR(_close_file_writers()); - const auto total_segment_num = _num_segment - _segcompacted_point + 1 + _num_segcompacted; RETURN_NOT_OK_STATUS_WITH_WARN(_check_segment_number_limit(total_segment_num), "too many segments when build new rowset"); @@ -741,12 +794,15 @@ Status BetaRowsetWriter::build(RowsetSharedPtr& rowset) { : _context.tablet_schema; _rowset_meta->set_tablet_schema(rowset_schema); - if (auto idx_files_info = _idx_files_info.get_inverted_files_info(_segment_start_id); - !idx_files_info.has_value()) [[unlikely]] { - LOG(ERROR) << "expected inverted index files info, but none presents: " - << idx_files_info.error(); - } else { - _rowset_meta->add_inverted_index_files_info(idx_files_info.value()); + // If segment compaction occurs, the idx file info will become inaccurate. + if (rowset_schema->has_inverted_index() && _num_segcompacted == 0) { + if (auto idx_files_info = _idx_files.inverted_index_file_info(_segment_start_id); + !idx_files_info.has_value()) [[unlikely]] { + LOG(ERROR) << "expected inverted index files info, but none presents: " + << idx_files_info.error(); + } else { + _rowset_meta->add_inverted_index_files_info(idx_files_info.value()); + } } RETURN_NOT_OK_STATUS_WITH_WARN(RowsetFactory::create_rowset(rowset_schema, _context.tablet_path, @@ -884,6 +940,14 @@ Status BaseBetaRowsetWriter::create_file_writer(uint32_t segment_id, io::FileWri fmt::format("failed to create file = {}, file type = {}", segment_path, file_type)); } +Status BaseBetaRowsetWriter::create_inverted_index_file_writer( + uint32_t segment_id, InvertedIndexFileWriterPtr* index_file_writer) { + RETURN_IF_ERROR(RowsetWriter::create_inverted_index_file_writer(segment_id, index_file_writer)); + // used for inverted index format v1 + (*index_file_writer)->set_file_writer_opts(_context.get_file_writer_options()); + return Status::OK(); +} + Status BetaRowsetWriter::_create_segment_writer_for_segcompaction( std::unique_ptr* writer, int64_t begin, int64_t end) { DCHECK(begin >= 0 && end >= 0); @@ -892,6 +956,22 @@ Status BetaRowsetWriter::_create_segment_writer_for_segcompaction( io::FileWriterPtr file_writer; RETURN_IF_ERROR(_create_file_writer(path, file_writer)); + InvertedIndexFileWriterPtr index_file_writer; + if (_context.tablet_schema->has_inverted_index()) { + io::FileWriterPtr idx_file_writer; + if (_context.tablet_schema->get_inverted_index_storage_format() != + InvertedIndexStorageFormatPB::V1) { + std::string prefix = + std::string {InvertedIndexDescriptor::get_index_file_path_prefix(path)}; + std::string index_path = InvertedIndexDescriptor::get_index_file_path_v2(prefix); + RETURN_IF_ERROR(_create_file_writer(index_path, idx_file_writer)); + } + index_file_writer = std::make_unique( + _context.fs(), path, _context.rowset_id.to_string(), _num_segcompacted, + _context.tablet_schema->get_inverted_index_storage_format(), + std::move(idx_file_writer)); + } + segment_v2::SegmentWriterOptions writer_options; writer_options.enable_unique_key_merge_on_write = _context.enable_unique_key_merge_on_write; writer_options.rowset_ctx = &_context; @@ -900,15 +980,14 @@ Status BetaRowsetWriter::_create_segment_writer_for_segcompaction( writer_options.max_rows_per_segment = _context.max_rows_per_segment; writer_options.mow_ctx = _context.mow_context; - *writer = std::make_unique(file_writer.get(), _num_segcompacted, - _context.tablet_schema, _context.tablet, - _context.data_dir, writer_options); + *writer = std::make_unique( + file_writer.get(), _num_segcompacted, _context.tablet_schema, _context.tablet, + _context.data_dir, writer_options, index_file_writer.get()); if (auto& seg_writer = _segcompaction_worker->get_file_writer(); seg_writer != nullptr && seg_writer->state() != io::FileWriter::State::CLOSED) { RETURN_IF_ERROR(_segcompaction_worker->get_file_writer()->close()); } _segcompaction_worker->get_file_writer().reset(file_writer.release()); - return Status::OK(); } @@ -998,11 +1077,13 @@ Status BetaRowsetWriter::flush_segment_writer_for_segcompaction( return Status::Error("failed to finalize segment: {}", s.to_string()); } + int64_t inverted_index_file_size = 0; + RETURN_IF_ERROR((*writer)->close_inverted_index(&inverted_index_file_size)); SegmentStatistics segstat; segstat.row_num = row_num; - segstat.data_size = segment_size + (*writer)->get_inverted_index_total_size(); - segstat.index_size = index_size + (*writer)->get_inverted_index_total_size(); + segstat.data_size = segment_size + inverted_index_file_size; + segstat.index_size = index_size + inverted_index_file_size; segstat.key_bounds = key_bounds; { std::lock_guard lock(_segid_statistics_map_mutex); diff --git a/be/src/olap/rowset/beta_rowset_writer.h b/be/src/olap/rowset/beta_rowset_writer.h index a83cf720d95e37..6063f7141771e4 100644 --- a/be/src/olap/rowset/beta_rowset_writer.h +++ b/be/src/olap/rowset/beta_rowset_writer.h @@ -42,6 +42,7 @@ #include "olap/rowset/rowset_writer.h" #include "olap/rowset/rowset_writer_context.h" #include "olap/rowset/segment_creator.h" +#include "segment_v2/inverted_index_file_writer.h" #include "segment_v2/segment.h" #include "util/spinlock.h" @@ -84,58 +85,33 @@ class SegmentFileCollection { bool _closed {false}; }; -// Collect the size of the inverted index files -class InvertedIndexFilesInfo { +class InvertedIndexFileCollection { public: + ~InvertedIndexFileCollection(); + + // `seg_id` -> inverted index file writer + Status add(int seg_id, InvertedIndexFileWriterPtr&& writer); + + // Close all file writers + // If the inverted index file writer is not closed, an error will be thrown during destruction + Status close(); + // Get inverted index file info in segment id order. - // Return the info of inverted index files from seg_id_offset to the last one. - Result> get_inverted_files_info(int seg_id_offset) { - std::lock_guard lock(_lock); - - Status st; - std::vector inverted_files_info(_inverted_index_files_info.size()); - bool succ = std::all_of( - _inverted_index_files_info.begin(), _inverted_index_files_info.end(), - [&](auto&& it) { - auto&& [seg_id, info] = it; - - int idx = seg_id - seg_id_offset; - if (idx >= inverted_files_info.size()) [[unlikely]] { - auto err_msg = fmt::format( - "invalid seg_id={} num_inverted_files_info={} seg_id_offset={}", - seg_id, inverted_files_info.size(), seg_id_offset); - DCHECK(false) << err_msg; - st = Status::InternalError(err_msg); - return false; - } - - auto& finfo = inverted_files_info[idx]; - if (finfo.has_index_size() || finfo.index_info_size() > 0) [[unlikely]] { - // File size should not been set - auto err_msg = fmt::format("duplicate seg_id={}", seg_id); - DCHECK(false) << err_msg; - st = Status::InternalError(err_msg); - return false; - } - finfo = info; - return true; - }); - - if (succ) { - return inverted_files_info; - } - - return ResultError(st); - } + // `seg_id_offset` is the offset of the segment id relative to the subscript of `_inverted_index_file_writers`, + // for more details, see `Tablet::create_transient_rowset_writer`. + Result> inverted_index_file_info(int seg_id_offset); - void add_file_info(int seg_id, InvertedIndexFileInfo file_info) { - std::lock_guard lock(_lock); - _inverted_index_files_info.emplace(seg_id, file_info); + // return all inverted index file writers + std::unordered_map& get_file_writers() { + return _inverted_index_file_writers; } + int64_t get_total_index_size() const { return _total_size; } + private: - std::unordered_map _inverted_index_files_info; mutable SpinLock _lock; + std::unordered_map _inverted_index_file_writers; + int64_t _total_size = 0; }; class BaseBetaRowsetWriter : public RowsetWriter { @@ -156,6 +132,9 @@ class BaseBetaRowsetWriter : public RowsetWriter { Status create_file_writer(uint32_t segment_id, io::FileWriterPtr& writer, FileType file_type = FileType::SEGMENT_FILE) override; + Status create_inverted_index_file_writer(uint32_t segment_id, + InvertedIndexFileWriterPtr* writer) override; + Status add_segment(uint32_t segment_id, const SegmentStatistics& segstat, TabletSchemaSPtr flush_schema) override; @@ -215,7 +194,9 @@ class BaseBetaRowsetWriter : public RowsetWriter { return _seg_files.get_file_writers(); } - InvertedIndexFilesInfo& get_inverted_index_files_info() { return _idx_files_info; } + std::unordered_map& inverted_index_file_writers() { + return this->_idx_files.get_file_writers(); + } private: void update_rowset_schema(TabletSchemaSPtr flush_schema); @@ -235,6 +216,16 @@ class BaseBetaRowsetWriter : public RowsetWriter { std::lock_guard l(_segid_statistics_map_mutex); return std::accumulate(_segment_num_rows.begin(), _segment_num_rows.end(), uint64_t(0)); } + // Only during vertical compaction is this method called + // Some index files are written during normal compaction and some files are written during index compaction. + // After all index writes are completed, call this method to write the final compound index file. + Status _close_inverted_index_file_writers() { + RETURN_NOT_OK_STATUS_WITH_WARN(_idx_files.close(), + "failed to close index file when build new rowset"); + this->_total_index_size += _idx_files.get_total_index_size(); + this->_total_data_size += _idx_files.get_total_index_size(); + return Status::OK(); + } std::atomic _num_segment; // number of consecutive flushed segments roaring::Roaring _segment_set; // bitmap set to record flushed segment id @@ -242,6 +233,7 @@ class BaseBetaRowsetWriter : public RowsetWriter { int32_t _segment_start_id; // basic write start from 0, partial update may be different SegmentFileCollection _seg_files; + InvertedIndexFileCollection _idx_files; // record rows number of every segment already written, using for rowid // conversion when compaction in unique key with MoW model @@ -269,9 +261,6 @@ class BaseBetaRowsetWriter : public RowsetWriter { int64_t _delete_bitmap_ns = 0; int64_t _segment_writer_ns = 0; - - // map - InvertedIndexFilesInfo _idx_files_info; }; class SegcompactionWorker; diff --git a/be/src/olap/rowset/beta_rowset_writer_v2.cpp b/be/src/olap/rowset/beta_rowset_writer_v2.cpp index 0d0ad435b9efd1..cb5dd5a5ee272d 100644 --- a/be/src/olap/rowset/beta_rowset_writer_v2.cpp +++ b/be/src/olap/rowset/beta_rowset_writer_v2.cpp @@ -58,7 +58,7 @@ namespace doris { using namespace ErrorCode; BetaRowsetWriterV2::BetaRowsetWriterV2(const std::vector>& streams) - : _segment_creator(_context, _seg_files, _idx_files_info), _streams(streams) {} + : _segment_creator(_context, _seg_files, _idx_files), _streams(streams) {} BetaRowsetWriterV2::~BetaRowsetWriterV2() = default; diff --git a/be/src/olap/rowset/beta_rowset_writer_v2.h b/be/src/olap/rowset/beta_rowset_writer_v2.h index 174b70a072bc17..cf658b38bda95a 100644 --- a/be/src/olap/rowset/beta_rowset_writer_v2.h +++ b/be/src/olap/rowset/beta_rowset_writer_v2.h @@ -154,11 +154,10 @@ class BetaRowsetWriterV2 : public RowsetWriter { std::vector _segments_encoded_key_bounds; SegmentFileCollection _seg_files; + InvertedIndexFileCollection _idx_files; SegmentCreator _segment_creator; - InvertedIndexFilesInfo _idx_files_info; - fmt::memory_buffer vlog_buffer; std::vector> _streams; diff --git a/be/src/olap/rowset/rowset_meta.cpp b/be/src/olap/rowset/rowset_meta.cpp index f053ad26d7efb9..fd9183fdac707a 100644 --- a/be/src/olap/rowset/rowset_meta.cpp +++ b/be/src/olap/rowset/rowset_meta.cpp @@ -266,20 +266,14 @@ InvertedIndexFileInfo RowsetMeta::inverted_index_file_info(int seg_id) { } void RowsetMeta::add_inverted_index_files_info( - const std::vector& idx_file_info) { + const std::vector& idx_file_info) { _rowset_meta_pb.set_enable_inverted_index_file_info(true); for (auto finfo : idx_file_info) { auto* new_file_info = _rowset_meta_pb.add_inverted_index_file_info(); - *new_file_info = finfo; + *new_file_info = *finfo; } } -void RowsetMeta::update_inverted_index_files_info( - const std::vector& idx_file_info) { - _rowset_meta_pb.clear_inverted_index_file_info(); - add_inverted_index_files_info(idx_file_info); -} - bool operator==(const RowsetMeta& a, const RowsetMeta& b) { if (a._rowset_id != b._rowset_id) return false; if (a._is_removed_from_rowset_meta != b._is_removed_from_rowset_meta) return false; diff --git a/be/src/olap/rowset/rowset_meta.h b/be/src/olap/rowset/rowset_meta.h index 4f25c676f6bd7f..ce414fcc7c087b 100644 --- a/be/src/olap/rowset/rowset_meta.h +++ b/be/src/olap/rowset/rowset_meta.h @@ -363,9 +363,8 @@ class RowsetMeta { return _rowset_meta_pb.inverted_index_file_info(); } - void add_inverted_index_files_info(const std::vector& idx_file_info); - - void update_inverted_index_files_info(const std::vector& idx_file_info); + void add_inverted_index_files_info( + const std::vector& idx_file_info); // Because the member field '_handle' is a raw pointer, use member func 'init' to replace copy ctor RowsetMeta(const RowsetMeta&) = delete; diff --git a/be/src/olap/rowset/rowset_writer.h b/be/src/olap/rowset/rowset_writer.h index 6861b8ab7e2ce6..ad42982488b316 100644 --- a/be/src/olap/rowset/rowset_writer.h +++ b/be/src/olap/rowset/rowset_writer.h @@ -31,6 +31,7 @@ #include "olap/column_mapping.h" #include "olap/rowset/rowset.h" #include "olap/rowset/rowset_writer_context.h" +#include "olap/rowset/segment_v2/inverted_index_file_writer.h" #include "olap/tablet_fwd.h" #include "olap/tablet_schema.h" #include "vec/core/block.h" @@ -95,6 +96,24 @@ class RowsetWriter { return Status::NotSupported("RowsetWriter does not support create_file_writer"); } + virtual Status create_inverted_index_file_writer( + uint32_t segment_id, InvertedIndexFileWriterPtr* index_file_writer) { + // Create file writer for the inverted index format v2. + io::FileWriterPtr idx_file_v2_ptr; + if (_context.tablet_schema->get_inverted_index_storage_format() != + InvertedIndexStorageFormatPB::V1) { + RETURN_IF_ERROR( + create_file_writer(segment_id, idx_file_v2_ptr, FileType::INVERTED_INDEX_FILE)); + } + std::string segment_prefix {InvertedIndexDescriptor::get_index_file_path_prefix( + _context.segment_path(segment_id))}; + *index_file_writer = std::make_unique( + _context.fs(), segment_prefix, _context.rowset_id.to_string(), segment_id, + _context.tablet_schema->get_inverted_index_storage_format(), + std::move(idx_file_v2_ptr)); + return Status::OK(); + } + // explicit flush all buffered rows into segment file. // note that `add_row` could also trigger flush when certain conditions are met virtual Status flush() = 0; diff --git a/be/src/olap/rowset/segment_creator.cpp b/be/src/olap/rowset/segment_creator.cpp index 1afd3215db42f6..5f4a3dce7b807e 100644 --- a/be/src/olap/rowset/segment_creator.cpp +++ b/be/src/olap/rowset/segment_creator.cpp @@ -53,8 +53,8 @@ namespace doris { using namespace ErrorCode; SegmentFlusher::SegmentFlusher(RowsetWriterContext& context, SegmentFileCollection& seg_files, - InvertedIndexFilesInfo& idx_files_info) - : _context(context), _seg_files(seg_files), _idx_files_info(idx_files_info) {} + InvertedIndexFileCollection& idx_files) + : _context(context), _seg_files(seg_files), _idx_files(idx_files) {} SegmentFlusher::~SegmentFlusher() = default; @@ -140,13 +140,10 @@ Status SegmentFlusher::_create_segment_writer(std::unique_ptrcreate(segment_id, segment_file_writer)); - io::FileWriterPtr inverted_file_writer; - if (_context.tablet_schema->has_inverted_index() && - _context.tablet_schema->get_inverted_index_storage_format() >= - InvertedIndexStorageFormatPB::V2 && - _context.memtable_on_sink_support_index_v2) { - RETURN_IF_ERROR(_context.file_writer_creator->create(segment_id, inverted_file_writer, - FileType::INVERTED_INDEX_FILE)); + InvertedIndexFileWriterPtr inverted_index_file_writer; + if (_context.tablet_schema->has_inverted_index()) { + RETURN_IF_ERROR( + _context.file_writer_creator->create(segment_id, &inverted_index_file_writer)); } segment_v2::SegmentWriterOptions writer_options; @@ -161,8 +158,11 @@ Status SegmentFlusher::_create_segment_writer(std::unique_ptr( segment_file_writer.get(), segment_id, _context.tablet_schema, _context.tablet, - _context.data_dir, writer_options, std::move(inverted_file_writer)); + _context.data_dir, writer_options, inverted_index_file_writer.get()); RETURN_IF_ERROR(_seg_files.add(segment_id, std::move(segment_file_writer))); + if (_context.tablet_schema->has_inverted_index()) { + RETURN_IF_ERROR(_idx_files.add(segment_id, std::move(inverted_index_file_writer))); + } auto s = writer->init(); if (!s.ok()) { LOG(WARNING) << "failed to init segment writer: " << s.to_string(); @@ -178,13 +178,10 @@ Status SegmentFlusher::_create_segment_writer( io::FileWriterPtr segment_file_writer; RETURN_IF_ERROR(_context.file_writer_creator->create(segment_id, segment_file_writer)); - io::FileWriterPtr inverted_file_writer; - if (_context.tablet_schema->has_inverted_index() && - _context.tablet_schema->get_inverted_index_storage_format() >= - InvertedIndexStorageFormatPB::V2 && - _context.memtable_on_sink_support_index_v2) { - RETURN_IF_ERROR(_context.file_writer_creator->create(segment_id, inverted_file_writer, - FileType::INVERTED_INDEX_FILE)); + InvertedIndexFileWriterPtr inverted_index_file_writer; + if (_context.tablet_schema->has_inverted_index()) { + RETURN_IF_ERROR( + _context.file_writer_creator->create(segment_id, &inverted_index_file_writer)); } segment_v2::VerticalSegmentWriterOptions writer_options; @@ -198,8 +195,11 @@ Status SegmentFlusher::_create_segment_writer( writer = std::make_unique( segment_file_writer.get(), segment_id, _context.tablet_schema, _context.tablet, - _context.data_dir, writer_options, std::move(inverted_file_writer)); + _context.data_dir, writer_options, inverted_index_file_writer.get()); RETURN_IF_ERROR(_seg_files.add(segment_id, std::move(segment_file_writer))); + if (_context.tablet_schema->has_inverted_index()) { + RETURN_IF_ERROR(_idx_files.add(segment_id, std::move(inverted_index_file_writer))); + } auto s = writer->init(); if (!s.ok()) { LOG(WARNING) << "failed to init segment writer: " << s.to_string(); @@ -231,6 +231,10 @@ Status SegmentFlusher::_flush_segment_writer( if (!s.ok()) { return Status::Error(s.code(), "failed to finalize segment: {}", s.to_string()); } + + int64_t inverted_index_file_size = 0; + RETURN_IF_ERROR(writer->close_inverted_index(&inverted_index_file_size)); + VLOG_DEBUG << "tablet_id:" << _context.tablet_id << " flushing filename: " << writer->data_dir_path() << " rowset_id:" << _context.rowset_id; @@ -245,17 +249,16 @@ Status SegmentFlusher::_flush_segment_writer( uint32_t segment_id = writer->segment_id(); SegmentStatistics segstat; segstat.row_num = row_num; - segstat.data_size = segment_size + writer->get_inverted_index_total_size(); - segstat.index_size = index_size + writer->get_inverted_index_total_size(); + segstat.data_size = segment_size + inverted_index_file_size; + segstat.index_size = index_size + inverted_index_file_size; segstat.key_bounds = key_bounds; - _idx_files_info.add_file_info(segment_id, writer->get_inverted_index_file_info()); writer.reset(); RETURN_IF_ERROR(_context.segment_collector->add(segment_id, segstat, flush_schema)); if (flush_size) { - *flush_size = segment_size + index_size; + *flush_size = segment_size + inverted_index_file_size; } return Status::OK(); } @@ -277,6 +280,10 @@ Status SegmentFlusher::_flush_segment_writer(std::unique_ptrclose_inverted_index(&inverted_index_file_size)); + VLOG_DEBUG << "tablet_id:" << _context.tablet_id << " flushing rowset_dir: " << _context.tablet_path << " rowset_id:" << _context.rowset_id; @@ -291,17 +298,16 @@ Status SegmentFlusher::_flush_segment_writer(std::unique_ptrget_segment_id(); SegmentStatistics segstat; segstat.row_num = row_num; - segstat.data_size = segment_size + writer->get_inverted_index_total_size(); - segstat.index_size = index_size + writer->get_inverted_index_total_size(); + segstat.data_size = segment_size + inverted_index_file_size; + segstat.index_size = index_size + inverted_index_file_size; segstat.key_bounds = key_bounds; - _idx_files_info.add_file_info(segment_id, writer->get_inverted_index_file_info()); writer.reset(); RETURN_IF_ERROR(_context.segment_collector->add(segment_id, segstat, flush_schema)); if (flush_size) { - *flush_size = segment_size + index_size; + *flush_size = segment_size + inverted_index_file_size; } return Status::OK(); } @@ -330,8 +336,8 @@ int64_t SegmentFlusher::Writer::max_row_to_add(size_t row_avg_size_in_bytes) { } SegmentCreator::SegmentCreator(RowsetWriterContext& context, SegmentFileCollection& seg_files, - InvertedIndexFilesInfo& idx_files_info) - : _segment_flusher(context, seg_files, idx_files_info) {} + InvertedIndexFileCollection& idx_files) + : _segment_flusher(context, seg_files, idx_files) {} Status SegmentCreator::add_block(const vectorized::Block* block) { if (block->rows() == 0) { diff --git a/be/src/olap/rowset/segment_creator.h b/be/src/olap/rowset/segment_creator.h index c862fce87a43bd..f8afd5798927d4 100644 --- a/be/src/olap/rowset/segment_creator.h +++ b/be/src/olap/rowset/segment_creator.h @@ -29,6 +29,7 @@ #include "io/fs/file_reader_writer_fwd.h" #include "olap/olap_common.h" #include "olap/rowset/rowset_writer_context.h" +#include "olap/rowset/segment_v2/inverted_index_file_writer.h" #include "olap/tablet_fwd.h" #include "util/spinlock.h" #include "vec/core/block.h" @@ -46,7 +47,7 @@ class VerticalSegmentWriter; struct SegmentStatistics; class BetaRowsetWriter; class SegmentFileCollection; -class InvertedIndexFilesInfo; +class InvertedIndexFileCollection; class FileWriterCreator { public: @@ -54,9 +55,12 @@ class FileWriterCreator { virtual Status create(uint32_t segment_id, io::FileWriterPtr& file_writer, FileType file_type = FileType::SEGMENT_FILE) = 0; + + virtual Status create(uint32_t segment_id, InvertedIndexFileWriterPtr* file_writer) = 0; }; template + requires std::is_base_of_v class FileWriterCreatorT : public FileWriterCreator { public: explicit FileWriterCreatorT(T* t) : _t(t) {} @@ -66,6 +70,10 @@ class FileWriterCreatorT : public FileWriterCreator { return _t->create_file_writer(segment_id, file_writer, file_type); } + Status create(uint32_t segment_id, InvertedIndexFileWriterPtr* file_writer) override { + return _t->create_inverted_index_file_writer(segment_id, file_writer); + } + private: T* _t = nullptr; }; @@ -79,6 +87,7 @@ class SegmentCollector { }; template + requires std::is_base_of_v class SegmentCollectorT : public SegmentCollector { public: explicit SegmentCollectorT(T* t) : _t(t) {} @@ -95,7 +104,7 @@ class SegmentCollectorT : public SegmentCollector { class SegmentFlusher { public: SegmentFlusher(RowsetWriterContext& context, SegmentFileCollection& seg_files, - InvertedIndexFilesInfo& idx_files_info); + InvertedIndexFileCollection& idx_files); ~SegmentFlusher(); @@ -164,7 +173,7 @@ class SegmentFlusher { private: RowsetWriterContext& _context; SegmentFileCollection& _seg_files; - InvertedIndexFilesInfo& _idx_files_info; + InvertedIndexFileCollection& _idx_files; // written rows by add_block/add_row std::atomic _num_rows_written = 0; @@ -177,7 +186,7 @@ class SegmentFlusher { class SegmentCreator { public: SegmentCreator(RowsetWriterContext& context, SegmentFileCollection& seg_files, - InvertedIndexFilesInfo& idx_files_info); + InvertedIndexFileCollection& idx_files); ~SegmentCreator() = default; diff --git a/be/src/olap/rowset/segment_v2/inverted_index_file_writer.cpp b/be/src/olap/rowset/segment_v2/inverted_index_file_writer.cpp index 7a784a55b862d0..70c1e55d1e8da8 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_file_writer.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_file_writer.cpp @@ -122,6 +122,8 @@ int64_t InvertedIndexFileWriter::headerLength() { } Status InvertedIndexFileWriter::close() { + DCHECK(!_closed) << debug_string(); + _closed = true; if (_indices_dirs.empty()) { return Status::OK(); } @@ -370,14 +372,10 @@ int64_t InvertedIndexFileWriter::write_v2() { out_dir->set_file_writer_opts(_opts); std::unique_ptr compound_file_output; - // idx v2 writer != nullptr means memtable on sink node now - if (_idx_v2_writer != nullptr) { - compound_file_output = std::unique_ptr( - out_dir->createOutputV2(_idx_v2_writer.get())); - } else { - compound_file_output = std::unique_ptr( - out_dir->createOutput(index_path.filename().c_str())); - } + + DCHECK(_idx_v2_writer != nullptr) << "inverted index file writer v2 is nullptr"; + compound_file_output = std::unique_ptr( + out_dir->createOutputV2(_idx_v2_writer.get())); // Write the version number compound_file_output->writeInt(InvertedIndexStorageFormatPB::V2); diff --git a/be/src/olap/rowset/segment_v2/inverted_index_file_writer.h b/be/src/olap/rowset/segment_v2/inverted_index_file_writer.h index 2aceb671d809a7..ccd6953cdd7abd 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_file_writer.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_file_writer.h @@ -38,6 +38,9 @@ class DorisFSDirectory; using InvertedIndexDirectoryMap = std::map, std::unique_ptr>; +class InvertedIndexFileWriter; +using InvertedIndexFileWriterPtr = std::unique_ptr; + class FileInfo { public: std::string filename; @@ -65,8 +68,14 @@ class InvertedIndexFileWriter { int64_t write_v1(); Status close(); int64_t headerLength(); - InvertedIndexFileInfo get_index_file_info() const { return _file_info; } - int64_t get_index_file_total_size() const { return _total_file_size; } + const InvertedIndexFileInfo* get_index_file_info() const { + DCHECK(_closed) << debug_string(); + return &_file_info; + } + int64_t get_index_file_total_size() const { + DCHECK(_closed) << debug_string(); + return _total_file_size; + } const io::FileSystemSPtr& get_fs() const { return _fs; } void sort_files(std::vector& file_infos); void copyFile(const char* fileName, lucene::store::Directory* dir, @@ -75,6 +84,20 @@ class InvertedIndexFileWriter { void set_file_writer_opts(const io::FileWriterOptions& opts) { _opts = opts; } + std::string debug_string() const { + std::stringstream indices_dirs; + for (const auto& [index, dir] : _indices_dirs) { + indices_dirs << "index id is: " << index.first << " , index suffix is: " << index.second + << " , index dir is: " << dir->toString(); + } + return fmt::format( + "inverted index file writer debug string: index storage format is: {}, index path " + "prefix is: {}, rowset id is: {}, seg id is: {}, closed is: {}, total file size " + "is: {}, index dirs is: {}", + _storage_format, _index_path_prefix, _rowset_id, _seg_id, _closed, _total_file_size, + indices_dirs.str()); + } + private: InvertedIndexDirectoryMap _indices_dirs; const io::FileSystemSPtr _fs; @@ -82,14 +105,18 @@ class InvertedIndexFileWriter { std::string _rowset_id; int64_t _seg_id; InvertedIndexStorageFormatPB _storage_format; - // v1: all file size - // v2: file size - int64_t _total_file_size = 0; + // write to disk or stream - io::FileWriterPtr _idx_v2_writer; + io::FileWriterPtr _idx_v2_writer = nullptr; io::FileWriterOptions _opts; + // v1: all file size + // v2: file size + int64_t _total_file_size = 0; InvertedIndexFileInfo _file_info; + + // only once + bool _closed = false; }; } // namespace segment_v2 } // namespace doris diff --git a/be/src/olap/rowset/segment_v2/segment_writer.cpp b/be/src/olap/rowset/segment_v2/segment_writer.cpp index b31ba22133a5e7..5957a555ba7857 100644 --- a/be/src/olap/rowset/segment_v2/segment_writer.cpp +++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp @@ -85,13 +85,14 @@ inline std::string segment_mem_tracker_name(uint32_t segment_id) { SegmentWriter::SegmentWriter(io::FileWriter* file_writer, uint32_t segment_id, TabletSchemaSPtr tablet_schema, BaseTabletSPtr tablet, DataDir* data_dir, const SegmentWriterOptions& opts, - io::FileWriterPtr inverted_file_writer) + InvertedIndexFileWriter* inverted_file_writer) : _segment_id(segment_id), _tablet_schema(std::move(tablet_schema)), _tablet(std::move(tablet)), _data_dir(data_dir), _opts(opts), _file_writer(file_writer), + _inverted_index_file_writer(inverted_file_writer), _mem_tracker(std::make_unique(segment_mem_tracker_name(segment_id))), _mow_context(std::move(opts.mow_ctx)) { CHECK_NOTNULL(file_writer); @@ -132,17 +133,6 @@ SegmentWriter::SegmentWriter(io::FileWriter* file_writer, uint32_t segment_id, } } } - if (_tablet_schema->has_inverted_index()) { - _inverted_index_file_writer = std::make_unique( - _opts.rowset_ctx->fs(), - std::string {InvertedIndexDescriptor::get_index_file_path_prefix( - file_writer->path().c_str())}, - _opts.rowset_ctx->rowset_id.to_string(), segment_id, - _tablet_schema->get_inverted_index_storage_format(), - std::move(inverted_file_writer)); - _inverted_index_file_writer->set_file_writer_opts( - _opts.rowset_ctx->get_file_writer_options()); - } } SegmentWriter::~SegmentWriter() { @@ -224,11 +214,12 @@ Status SegmentWriter::_create_column_writer(uint32_t cid, const TabletColumn& co opts.need_bloom_filter = false; opts.need_bitmap_index = false; } - opts.inverted_index_file_writer = _inverted_index_file_writer.get(); + opts.inverted_index_file_writer = _inverted_index_file_writer; for (const auto* index : opts.indexes) { if (!skip_inverted_index && index->index_type() == IndexType::INVERTED) { opts.inverted_index = index; opts.need_inverted_index = true; + DCHECK(_inverted_index_file_writer != nullptr); // TODO support multiple inverted index break; } @@ -1011,10 +1002,6 @@ Status SegmentWriter::finalize_footer(uint64_t* segment_file_size) { if (*segment_file_size == 0) { return Status::Corruption("Bad segment, file size = 0"); } - if (_inverted_index_file_writer != nullptr) { - RETURN_IF_ERROR(_inverted_index_file_writer->close()); - _inverted_index_file_info = _inverted_index_file_writer->get_index_file_info(); - } return Status::OK(); } @@ -1255,13 +1242,6 @@ Status SegmentWriter::_generate_short_key_index( return Status::OK(); } -int64_t SegmentWriter::get_inverted_index_total_size() { - if (_inverted_index_file_writer != nullptr) { - return _inverted_index_file_writer->get_index_file_total_size(); - } - return 0; -} - inline bool SegmentWriter::_is_mow() { return _tablet_schema->keys_type() == UNIQUE_KEYS && _opts.enable_unique_key_merge_on_write; } diff --git a/be/src/olap/rowset/segment_v2/segment_writer.h b/be/src/olap/rowset/segment_v2/segment_writer.h index 37b514e69c7001..4b157092221d49 100644 --- a/be/src/olap/rowset/segment_v2/segment_writer.h +++ b/be/src/olap/rowset/segment_v2/segment_writer.h @@ -33,6 +33,7 @@ #include "gutil/strings/substitute.h" #include "olap/olap_define.h" #include "olap/rowset/segment_v2/column_writer.h" +#include "olap/rowset/segment_v2/inverted_index_file_writer.h" #include "olap/tablet.h" #include "olap/tablet_schema.h" #include "util/faststring.h" @@ -60,7 +61,6 @@ class FileWriter; } // namespace io namespace segment_v2 { -class InvertedIndexFileWriter; extern const char* k_segment_magic; extern const uint32_t k_segment_magic_length; @@ -83,7 +83,7 @@ class SegmentWriter { explicit SegmentWriter(io::FileWriter* file_writer, uint32_t segment_id, TabletSchemaSPtr tablet_schema, BaseTabletSPtr tablet, DataDir* data_dir, const SegmentWriterOptions& opts, - io::FileWriterPtr inverted_file_writer = nullptr); + InvertedIndexFileWriter* inverted_file_writer); ~SegmentWriter(); Status init(); @@ -110,9 +110,6 @@ class SegmentWriter { uint64_t estimate_segment_size(); - InvertedIndexFileInfo get_inverted_index_file_info() const { return _inverted_index_file_info; } - int64_t get_inverted_index_total_size(); - uint32_t num_rows_written() const { return _num_rows_written; } // for partial update @@ -144,6 +141,17 @@ class SegmentWriter { void set_mow_context(std::shared_ptr mow_context); + Status close_inverted_index(int64_t* inverted_index_file_size) { + // no inverted index + if (_inverted_index_file_writer == nullptr) { + *inverted_index_file_size = 0; + return Status::OK(); + } + RETURN_IF_ERROR(_inverted_index_file_writer->close()); + *inverted_index_file_size = _inverted_index_file_writer->get_index_file_total_size(); + return Status::OK(); + } + private: DISALLOW_COPY_AND_ASSIGN(SegmentWriter); Status _create_column_writer(uint32_t cid, const TabletColumn& column, @@ -199,13 +207,15 @@ class SegmentWriter { // Not owned. owned by RowsetWriter or SegmentFlusher io::FileWriter* _file_writer = nullptr; - std::unique_ptr _inverted_index_file_writer; + // Not owned. owned by RowsetWriter or SegmentFlusher + InvertedIndexFileWriter* _inverted_index_file_writer = nullptr; + SegmentFooterPB _footer; // for mow tables with cluster key, the sort key is the cluster keys not unique keys // for other tables, the sort key is the keys size_t _num_sort_key_columns; size_t _num_short_key_columns; - InvertedIndexFileInfo _inverted_index_file_info; + std::unique_ptr _short_key_index_builder; std::unique_ptr _primary_key_index_builder; std::vector> _column_writers; diff --git a/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp b/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp index 4f94189a6212eb..9ade9c1bfccb89 100644 --- a/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp +++ b/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp @@ -88,13 +88,14 @@ VerticalSegmentWriter::VerticalSegmentWriter(io::FileWriter* file_writer, uint32 TabletSchemaSPtr tablet_schema, BaseTabletSPtr tablet, DataDir* data_dir, const VerticalSegmentWriterOptions& opts, - io::FileWriterPtr inverted_file_writer) + InvertedIndexFileWriter* inverted_file_writer) : _segment_id(segment_id), _tablet_schema(std::move(tablet_schema)), _tablet(std::move(tablet)), _data_dir(data_dir), _opts(opts), _file_writer(file_writer), + _inverted_index_file_writer(inverted_file_writer), _mem_tracker(std::make_unique( vertical_segment_writer_mem_tracker_name(segment_id))), _mow_context(std::move(opts.mow_ctx)) { @@ -136,17 +137,6 @@ VerticalSegmentWriter::VerticalSegmentWriter(io::FileWriter* file_writer, uint32 } } } - if (_tablet_schema->has_inverted_index()) { - _inverted_index_file_writer = std::make_unique( - _opts.rowset_ctx->fs(), - std::string {InvertedIndexDescriptor::get_index_file_path_prefix( - _opts.rowset_ctx->segment_path(segment_id))}, - _opts.rowset_ctx->rowset_id.to_string(), segment_id, - _tablet_schema->get_inverted_index_storage_format(), - std::move(inverted_file_writer)); - _inverted_index_file_writer->set_file_writer_opts( - _opts.rowset_ctx->get_file_writer_options()); - } } VerticalSegmentWriter::~VerticalSegmentWriter() { @@ -220,11 +210,12 @@ Status VerticalSegmentWriter::_create_column_writer(uint32_t cid, const TabletCo if (!skip_inverted_index && index->index_type() == IndexType::INVERTED) { opts.inverted_index = index; opts.need_inverted_index = true; + DCHECK(_inverted_index_file_writer != nullptr); // TODO support multiple inverted index break; } } - opts.inverted_index_file_writer = _inverted_index_file_writer.get(); + opts.inverted_index_file_writer = _inverted_index_file_writer; #define CHECK_FIELD_TYPE(TYPE, type_name) \ if (column.type() == FieldType::OLAP_FIELD_TYPE_##TYPE) { \ @@ -996,9 +987,6 @@ Status VerticalSegmentWriter::finalize_columns_index(uint64_t* index_size) { *index_size = _file_writer->bytes_appended() - index_start; } - if (_inverted_index_file_writer != nullptr) { - _inverted_index_file_info = _inverted_index_file_writer->get_index_file_info(); - } // reset all column writers and data_conveter clear(); @@ -1073,9 +1061,6 @@ Status VerticalSegmentWriter::_write_inverted_index() { for (auto& column_writer : _column_writers) { RETURN_IF_ERROR(column_writer->write_inverted_index()); } - if (_inverted_index_file_writer != nullptr) { - RETURN_IF_ERROR(_inverted_index_file_writer->close()); - } return Status::OK(); } @@ -1162,13 +1147,6 @@ void VerticalSegmentWriter::_set_max_key(const Slice& key) { _max_key.append(key.get_data(), key.get_size()); } -int64_t VerticalSegmentWriter::get_inverted_index_total_size() { - if (_inverted_index_file_writer != nullptr) { - return _inverted_index_file_writer->get_index_file_total_size(); - } - return 0; -} - inline bool VerticalSegmentWriter::_is_mow() { return _tablet_schema->keys_type() == UNIQUE_KEYS && _opts.enable_unique_key_merge_on_write; } diff --git a/be/src/olap/rowset/segment_v2/vertical_segment_writer.h b/be/src/olap/rowset/segment_v2/vertical_segment_writer.h index 56102c5d58d32b..0e04b49b343709 100644 --- a/be/src/olap/rowset/segment_v2/vertical_segment_writer.h +++ b/be/src/olap/rowset/segment_v2/vertical_segment_writer.h @@ -34,6 +34,7 @@ #include "gutil/strings/substitute.h" #include "olap/olap_define.h" #include "olap/rowset/segment_v2/column_writer.h" +#include "olap/rowset/segment_v2/inverted_index_file_writer.h" #include "olap/tablet.h" #include "olap/tablet_schema.h" #include "util/faststring.h" @@ -82,7 +83,7 @@ class VerticalSegmentWriter { explicit VerticalSegmentWriter(io::FileWriter* file_writer, uint32_t segment_id, TabletSchemaSPtr tablet_schema, BaseTabletSPtr tablet, DataDir* data_dir, const VerticalSegmentWriterOptions& opts, - io::FileWriterPtr inverted_file_writer = nullptr); + InvertedIndexFileWriter* inverted_file_writer); ~VerticalSegmentWriter(); VerticalSegmentWriter(const VerticalSegmentWriter&) = delete; @@ -99,9 +100,7 @@ class VerticalSegmentWriter { [[nodiscard]] std::string data_dir_path() const { return _data_dir == nullptr ? "" : _data_dir->path(); } - [[nodiscard]] InvertedIndexFileInfo get_inverted_index_file_info() const { - return _inverted_index_file_info; - } + [[nodiscard]] uint32_t num_rows_written() const { return _num_rows_written; } // for partial update @@ -122,10 +121,19 @@ class VerticalSegmentWriter { TabletSchemaSPtr flush_schema() const { return _flush_schema; }; - int64_t get_inverted_index_total_size(); - void clear(); + Status close_inverted_index(int64_t* inverted_index_file_size) { + // no inverted index + if (_inverted_index_file_writer == nullptr) { + *inverted_index_file_size = 0; + return Status::OK(); + } + RETURN_IF_ERROR(_inverted_index_file_writer->close()); + *inverted_index_file_size = _inverted_index_file_writer->get_index_file_total_size(); + return Status::OK(); + } + private: void _init_column_meta(ColumnMetaPB* meta, uint32_t column_id, const TabletColumn& column); Status _create_column_writer(uint32_t cid, const TabletColumn& column, @@ -189,14 +197,15 @@ class VerticalSegmentWriter { // Not owned. owned by RowsetWriter io::FileWriter* _file_writer = nullptr; - std::unique_ptr _inverted_index_file_writer; + // Not owned. owned by RowsetWriter or SegmentFlusher + InvertedIndexFileWriter* _inverted_index_file_writer = nullptr; SegmentFooterPB _footer; // for mow tables with cluster key, the sort key is the cluster keys not unique keys // for other tables, the sort key is the keys size_t _num_sort_key_columns; size_t _num_short_key_columns; - InvertedIndexFileInfo _inverted_index_file_info; + std::unique_ptr _short_key_index_builder; std::unique_ptr _primary_key_index_builder; std::vector> _column_writers; diff --git a/be/src/olap/rowset/vertical_beta_rowset_writer.cpp b/be/src/olap/rowset/vertical_beta_rowset_writer.cpp index ced0fb880c41fb..fb8f6622685856 100644 --- a/be/src/olap/rowset/vertical_beta_rowset_writer.cpp +++ b/be/src/olap/rowset/vertical_beta_rowset_writer.cpp @@ -164,26 +164,28 @@ Status VerticalBetaRowsetWriter::_create_segment_writer( int seg_id = this->_num_segment.fetch_add(1, std::memory_order_relaxed); - io::FileWriterPtr file_writer; - io::FileWriterOptions opts = this->_context.get_file_writer_options(); - - auto path = context.segment_path(seg_id); - auto& fs = context.fs_ref(); - Status st = fs.create_file(path, &file_writer, &opts); - if (!st.ok()) { - LOG(WARNING) << "failed to create writable file. path=" << path << ", err: " << st; - return st; + io::FileWriterPtr segment_file_writer; + RETURN_IF_ERROR(BaseBetaRowsetWriter::create_file_writer(seg_id, segment_file_writer)); + DCHECK(segment_file_writer != nullptr); + + InvertedIndexFileWriterPtr inverted_index_file_writer; + if (context.tablet_schema->has_inverted_index()) { + RETURN_IF_ERROR(RowsetWriter::create_inverted_index_file_writer( + seg_id, &inverted_index_file_writer)); } - DCHECK(file_writer != nullptr); segment_v2::SegmentWriterOptions writer_options; writer_options.enable_unique_key_merge_on_write = context.enable_unique_key_merge_on_write; writer_options.rowset_ctx = &context; writer_options.max_rows_per_segment = context.max_rows_per_segment; - *writer = std::make_unique(file_writer.get(), seg_id, - context.tablet_schema, context.tablet, - context.data_dir, writer_options); - RETURN_IF_ERROR(this->_seg_files.add(seg_id, std::move(file_writer))); + *writer = std::make_unique( + segment_file_writer.get(), seg_id, context.tablet_schema, context.tablet, + context.data_dir, writer_options, inverted_index_file_writer.get()); + + RETURN_IF_ERROR(this->_seg_files.add(seg_id, std::move(segment_file_writer))); + if (context.tablet_schema->has_inverted_index()) { + RETURN_IF_ERROR(this->_idx_files.add(seg_id, std::move(inverted_index_file_writer))); + } auto s = (*writer)->init(column_ids, is_key); if (!s.ok()) { @@ -205,10 +207,7 @@ Status VerticalBetaRowsetWriter::final_flush() { LOG(WARNING) << "Fail to finalize segment footer, " << st; return st; } - this->_total_data_size += segment_size + segment_writer->get_inverted_index_total_size(); - this->_total_index_size += segment_writer->get_inverted_index_total_size(); - this->_idx_files_info.add_file_info(segment_writer->get_segment_id(), - segment_writer->get_inverted_index_file_info()); + this->_total_data_size += segment_size; segment_writer.reset(); } return Status::OK(); @@ -217,6 +216,7 @@ Status VerticalBetaRowsetWriter::final_flush() { template requires std::is_base_of_v Status VerticalBetaRowsetWriter::_close_file_writers() { + RETURN_IF_ERROR(BaseBetaRowsetWriter::_close_inverted_index_file_writers()); return this->_seg_files.close(); } diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp index 468a6b2fb126f0..8c1939152af8ee 100644 --- a/be/src/olap/tablet_manager.cpp +++ b/be/src/olap/tablet_manager.cpp @@ -101,7 +101,9 @@ TabletManager::TabletManager(StorageEngine& engine, int32_t tablet_map_lock_shar } TabletManager::~TabletManager() { +#ifndef BE_TEST DEREGISTER_HOOK_METRIC(tablet_meta_mem_consumption); +#endif } Status TabletManager::_add_tablet_unlocked(TTabletId tablet_id, const TabletSharedPtr& tablet, diff --git a/be/src/olap/tablet_meta.h b/be/src/olap/tablet_meta.h index a453baf745d602..cb329bde1e0c4a 100644 --- a/be/src/olap/tablet_meta.h +++ b/be/src/olap/tablet_meta.h @@ -118,6 +118,11 @@ class TabletMeta { TabletMeta(const TabletMeta& tablet_meta); TabletMeta(TabletMeta&& tablet_meta) = delete; +// UT +#ifdef BE_TEST + TabletMeta(TabletSchemaSPtr tablet_schema) : _schema(tablet_schema) {} +#endif + // Function create_from_file is used to be compatible with previous tablet_meta. // Previous tablet_meta is a physical file in tablet dir, which is not stored in rocksdb. Status create_from_file(const std::string& file_path); diff --git a/be/src/olap/task/index_builder.cpp b/be/src/olap/task/index_builder.cpp index 3af35ee78a84d9..7e6df26f683dab 100644 --- a/be/src/olap/task/index_builder.cpp +++ b/be/src/olap/task/index_builder.cpp @@ -298,10 +298,20 @@ Status IndexBuilder::handle_single_rowset(RowsetMetaSharedPtr output_rowset_meta _tablet->tablet_path(), output_rowset_meta->rowset_id().to_string(), seg_ptr->id()))}; + std::string index_path = + InvertedIndexDescriptor::get_index_file_path_v2(index_path_prefix); + io::FileWriterPtr file_writer; + Status st = fs->create_file(index_path, &file_writer); + if (!st.ok()) { + LOG(WARNING) << "failed to create writable file. path=" << index_path + << ", err: " << st; + return st; + } auto inverted_index_file_writer = std::make_unique( fs, std::move(index_path_prefix), output_rowset_meta->rowset_id().to_string(), seg_ptr->id(), - output_rowset_schema->get_inverted_index_storage_format()); + output_rowset_schema->get_inverted_index_storage_format(), + std::move(file_writer)); RETURN_IF_ERROR(inverted_index_file_writer->initialize(dirs)); // create inverted index writer for (auto& index_meta : _dropped_inverted_indexes) { @@ -352,10 +362,20 @@ Status IndexBuilder::handle_single_rowset(RowsetMetaSharedPtr output_rowset_meta << seg_ptr->id() << " cannot be found"; continue; } + std::string index_path = + InvertedIndexDescriptor::get_index_file_path_v2(index_path_prefix); + io::FileWriterPtr file_writer; + Status st = fs->create_file(index_path, &file_writer); + if (!st.ok()) { + LOG(WARNING) << "failed to create writable file. path=" << index_path + << ", err: " << st; + return st; + } auto dirs = DORIS_TRY(idx_file_reader_iter->second->get_all_directories()); inverted_index_file_writer = std::make_unique( fs, index_path_prefix, output_rowset_meta->rowset_id().to_string(), - seg_ptr->id(), output_rowset_schema->get_inverted_index_storage_format()); + seg_ptr->id(), output_rowset_schema->get_inverted_index_storage_format(), + std::move(file_writer)); RETURN_IF_ERROR(inverted_index_file_writer->initialize(dirs)); } else { inverted_index_file_writer = std::make_unique( diff --git a/be/test/olap/delete_bitmap_calculator_test.cpp b/be/test/olap/delete_bitmap_calculator_test.cpp index aec1b52087d592..ee54a061363d5c 100644 --- a/be/test/olap/delete_bitmap_calculator_test.cpp +++ b/be/test/olap/delete_bitmap_calculator_test.cpp @@ -103,7 +103,8 @@ class DeleteBitmapCalculatorTest : public testing::Test { io::FileWriterPtr file_writer; Status st = fs->create_file(path, &file_writer); EXPECT_TRUE(st.ok()); - SegmentWriter writer(file_writer.get(), segment_id, build_schema, nullptr, nullptr, opts); + SegmentWriter writer(file_writer.get(), segment_id, build_schema, nullptr, nullptr, opts, + nullptr); st = writer.init(); EXPECT_TRUE(st.ok()); diff --git a/be/test/olap/rowset/segment_v2/inverted_index/compaction/index_compaction_test.cpp b/be/test/olap/rowset/segment_v2/inverted_index/compaction/index_compaction_test.cpp new file mode 100644 index 00000000000000..922a77fcaa4e47 --- /dev/null +++ b/be/test/olap/rowset/segment_v2/inverted_index/compaction/index_compaction_test.cpp @@ -0,0 +1,443 @@ +// 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 "olap/cumulative_compaction.h" +#include "olap/rowset/beta_rowset_writer.h" +#include "olap/rowset/rowset_factory.h" +#include "olap/rowset/segment_v2/inverted_index/query/query_factory.h" +#include "olap/rowset/segment_v2/inverted_index_file_reader.h" +#include "olap/storage_engine.h" + +namespace doris { + +using namespace doris::vectorized; + +constexpr static uint32_t MAX_PATH_LEN = 1024; +constexpr static std::string_view dest_dir = "./ut_dir/inverted_index_test"; +constexpr static std::string_view tmp_dir = "./ut_dir/tmp"; +static int64_t inc_id = 1000; + +struct DataRow { + int key; + std::string word; + std::string url; + int num; +}; + +class IndexCompactionTest : public ::testing::Test { +protected: + void SetUp() override { + // absolute dir + char buffer[MAX_PATH_LEN]; + EXPECT_NE(getcwd(buffer, MAX_PATH_LEN), nullptr); + _curreent_dir = std::string(buffer); + _absolute_dir = _curreent_dir + std::string(dest_dir); + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_absolute_dir).ok()); + EXPECT_TRUE(io::global_local_filesystem()->create_directory(_absolute_dir).ok()); + + // tmp dir + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(tmp_dir).ok()); + EXPECT_TRUE(io::global_local_filesystem()->create_directory(tmp_dir).ok()); + std::vector paths; + paths.emplace_back(std::string(tmp_dir), 1024000000); + auto tmp_file_dirs = std::make_unique(paths); + EXPECT_TRUE(tmp_file_dirs->init().ok()); + ExecEnv::GetInstance()->set_tmp_file_dir(std::move(tmp_file_dirs)); + + // storage engine + doris::EngineOptions options; + auto engine = std::make_unique(options); + _engine_ref = engine.get(); + _data_dir = std::make_unique(*_engine_ref, _absolute_dir); + static_cast(_data_dir->update_capacity()); + ExecEnv::GetInstance()->set_storage_engine(std::move(engine)); + + // tablet_schema + TabletSchemaPB schema_pb; + schema_pb.set_keys_type(KeysType::DUP_KEYS); + schema_pb.set_inverted_index_storage_format(InvertedIndexStorageFormatPB::V2); + + construct_column(schema_pb.add_column(), schema_pb.add_index(), 10000, "key_index", 0, + "INT", "key"); + construct_column(schema_pb.add_column(), schema_pb.add_index(), 10001, "v1_index", 1, + "STRING", "v1"); + construct_column(schema_pb.add_column(), schema_pb.add_index(), 10002, "v2_index", 2, + "STRING", "v2", true); + construct_column(schema_pb.add_column(), schema_pb.add_index(), 10003, "v3_index", 3, "INT", + "v3"); + + _tablet_schema.reset(new TabletSchema); + _tablet_schema->init_from_pb(schema_pb); + + // tablet + TabletMetaSharedPtr tablet_meta(new TabletMeta(_tablet_schema)); + + _tablet.reset(new Tablet(*_engine_ref, tablet_meta, _data_dir.get())); + EXPECT_TRUE(_tablet->init().ok()); + config::inverted_index_compaction_enable = true; + } + void TearDown() override { + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_tablet->tablet_path()).ok()); + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_absolute_dir).ok()); + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(tmp_dir).ok()); + _engine_ref = nullptr; + ExecEnv::GetInstance()->set_storage_engine(nullptr); + } + + void construct_column(ColumnPB* column_pb, TabletIndexPB* tablet_index, int64_t index_id, + const std::string& index_name, int32_t col_unique_id, + const std::string& column_type, const std::string& column_name, + bool parser = false) { + column_pb->set_unique_id(col_unique_id); + column_pb->set_name(column_name); + column_pb->set_type(column_type); + column_pb->set_is_key(false); + column_pb->set_is_nullable(true); + tablet_index->set_index_id(index_id); + tablet_index->set_index_name(index_name); + tablet_index->set_index_type(IndexType::INVERTED); + tablet_index->add_col_unique_id(col_unique_id); + if (parser) { + auto* properties = tablet_index->mutable_properties(); + (*properties)[INVERTED_INDEX_PARSER_KEY] = INVERTED_INDEX_PARSER_UNICODE; + } + } + + RowsetWriterContext rowset_writer_context() { + RowsetWriterContext context; + RowsetId rowset_id; + rowset_id.init(inc_id); + context.rowset_id = rowset_id; + context.rowset_type = BETA_ROWSET; + context.data_dir = _data_dir.get(); + context.rowset_state = VISIBLE; + context.tablet_schema = _tablet_schema; + context.tablet_path = _tablet->tablet_path(); + context.version = Version(inc_id, inc_id); + context.max_rows_per_segment = 200; + inc_id++; + return context; + } + + IndexCompactionTest() = default; + ~IndexCompactionTest() override = default; + +private: + TabletSchemaSPtr _tablet_schema = nullptr; + StorageEngine* _engine_ref = nullptr; + std::unique_ptr _data_dir = nullptr; + TabletSharedPtr _tablet = nullptr; + std::string _absolute_dir; + std::string _curreent_dir; +}; + +std::vector read_data(const std::string file_name) { + std::ifstream file(file_name); + EXPECT_TRUE(file.is_open()); + + std::string line; + std::vector data; + + while (std::getline(file, line)) { + std::stringstream ss(line); + std::string item; + DataRow row; + EXPECT_TRUE(std::getline(ss, item, ',')); + row.key = std::stoi(item); + EXPECT_TRUE(std::getline(ss, item, ',')); + row.word = item; + EXPECT_TRUE(std::getline(ss, item, ',')); + row.url = item; + EXPECT_TRUE(std::getline(ss, item, ',')); + row.num = std::stoi(item); + data.emplace_back(std::move(row)); + } + + file.close(); + return data; +} + +bool query_bkd(const TabletIndex* index, + std::shared_ptr& inverted_index_file_reader, + const std::vector& query_data, const std::vector& query_result) { + const auto& idx_reader = BkdIndexReader::create_shared(index, inverted_index_file_reader); + const auto& index_searcher_builder = std::make_unique(); + auto dir = inverted_index_file_reader->open(index); + EXPECT_TRUE(dir.has_value()); + auto searcher_result = index_searcher_builder->get_index_searcher(dir.value().release()); + EXPECT_TRUE(searcher_result.has_value()); + auto bkd_searcher = std::get_if(&searcher_result.value()); + EXPECT_TRUE(bkd_searcher != nullptr); + idx_reader->_type_info = get_scalar_type_info((FieldType)(*bkd_searcher)->type); + EXPECT_TRUE(idx_reader->_type_info != nullptr); + idx_reader->_value_key_coder = get_key_coder(idx_reader->_type_info->type()); + + for (int i = 0; i < query_data.size(); i++) { + vectorized::Field param_value = Int32(query_data[i]); + std::unique_ptr query_param = nullptr; + EXPECT_TRUE(segment_v2::InvertedIndexQueryParamFactory::create_query_value( + PrimitiveType::TYPE_INT, ¶m_value, query_param) + .ok()); + auto result = std::make_shared(); + EXPECT_TRUE(idx_reader + ->invoke_bkd_query(query_param->get_value(), + InvertedIndexQueryType::EQUAL_QUERY, *bkd_searcher, + result) + .ok()); + EXPECT_EQ(query_result[i], result->cardinality()) << query_data[i]; + } + return true; +} + +bool query_string(const TabletIndex* index, + std::shared_ptr& inverted_index_file_reader, + const std::string& column_name, const std::vector& query_data, + const std::vector& query_result) { + const auto& idx_reader = + StringTypeInvertedIndexReader::create_shared(index, inverted_index_file_reader); + const auto& index_searcher_builder = std::make_unique(); + auto dir = inverted_index_file_reader->open(index); + EXPECT_TRUE(dir.has_value()); + auto searcher_result = index_searcher_builder->get_index_searcher(dir.value().release()); + EXPECT_TRUE(searcher_result.has_value()); + auto string_searcher = std::get_if(&searcher_result.value()); + EXPECT_TRUE(string_searcher != nullptr); + std::wstring column_name_ws = StringUtil::string_to_wstring(column_name); + + for (int i = 0; i < query_data.size(); i++) { + TQueryOptions queryOptions; + auto query = QueryFactory::create(InvertedIndexQueryType::EQUAL_QUERY, *string_searcher, + queryOptions); + EXPECT_TRUE(query != nullptr); + InvertedIndexQueryInfo query_info; + query_info.field_name = column_name_ws; + query_info.terms.emplace_back(query_data[i]); + query->add(query_info); + auto result = std::make_shared(); + query->search(*result); + EXPECT_EQ(query_result[i], result->cardinality()) << query_data[i]; + } + return true; +} + +bool query_fulltext(const TabletIndex* index, + std::shared_ptr& inverted_index_file_reader, + const std::string& column_name, const std::vector& query_data, + const std::vector& query_result) { + const auto& idx_reader = FullTextIndexReader::create_shared(index, inverted_index_file_reader); + const auto& index_searcher_builder = std::make_unique(); + auto dir = inverted_index_file_reader->open(index); + EXPECT_TRUE(dir.has_value()); + auto searcher_result = index_searcher_builder->get_index_searcher(dir.value().release()); + EXPECT_TRUE(searcher_result.has_value()); + auto string_searcher = std::get_if(&searcher_result.value()); + EXPECT_TRUE(string_searcher != nullptr); + std::wstring column_name_ws = StringUtil::string_to_wstring(column_name); + + for (int i = 0; i < query_data.size(); i++) { + TQueryOptions queryOptions; + auto query = QueryFactory::create(InvertedIndexQueryType::MATCH_ANY_QUERY, *string_searcher, + queryOptions); + EXPECT_TRUE(query != nullptr); + InvertedIndexQueryInfo query_info; + query_info.field_name = column_name_ws; + query_info.terms.emplace_back(query_data[i]); + query->add(query_info); + auto result = std::make_shared(); + query->search(*result); + EXPECT_EQ(query_result[i], result->cardinality()) << query_data[i]; + } + return true; +} + +TEST_F(IndexCompactionTest, write_index_test) { + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_tablet->tablet_path()).ok()); + EXPECT_TRUE(io::global_local_filesystem()->create_directory(_tablet->tablet_path()).ok()); + std::string data_file1 = + _curreent_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv"; + std::string data_file2 = + _curreent_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv"; + + std::vector> data; + data.emplace_back(read_data(data_file1)); + data.emplace_back(read_data(data_file2)); + + std::vector rowsets(data.size()); + for (int i = 0; i < data.size(); i++) { + const auto& res = + RowsetFactory::create_rowset_writer(*_engine_ref, rowset_writer_context(), false); + EXPECT_TRUE(res.has_value()) << res.error(); + const auto& rowset_writer = res.value(); + + Block block = _tablet_schema->create_block(); + auto columns = block.mutate_columns(); + for (const auto& row : data[i]) { + vectorized::Field key = Int32(row.key); + vectorized::Field v1 = row.word; + vectorized::Field v2 = row.url; + vectorized::Field v3 = Int32(row.num); + columns[0]->insert(key); + columns[1]->insert(v1); + columns[2]->insert(v2); + columns[3]->insert(v3); + } + EXPECT_TRUE(rowset_writer->add_block(&block).ok()); + EXPECT_TRUE(rowset_writer->flush().ok()); + const auto& dst_writer = dynamic_cast(rowset_writer.get()); + + // inverted index file writer + for (const auto& [seg_id, idx_file_writer] : dst_writer->_idx_files.get_file_writers()) { + EXPECT_TRUE(idx_file_writer->_closed); + } + + EXPECT_TRUE(rowset_writer->build(rowsets[i]).ok()); + EXPECT_TRUE(_tablet->add_rowset(rowsets[i]).ok()); + EXPECT_TRUE(rowsets[i]->num_segments() == 5); + + // check rowset meta and file + for (int seg_id = 0; seg_id < rowsets[i]->num_segments(); seg_id++) { + const auto& index_info = rowsets[i]->_rowset_meta->inverted_index_file_info(seg_id); + EXPECT_TRUE(index_info.has_index_size()); + const auto& fs = rowsets[i]->_rowset_meta->fs(); + const auto& file_name = fmt::format("{}/{}_{}.idx", rowsets[i]->tablet_path(), + rowsets[i]->rowset_id().to_string(), seg_id); + int64_t file_size = 0; + EXPECT_TRUE(fs->file_size(file_name, &file_size).ok()); + EXPECT_EQ(index_info.index_size(), file_size); + + const auto& seg_path = rowsets[i]->segment_path(seg_id); + EXPECT_TRUE(seg_path.has_value()); + const auto& index_file_path_prefix = + InvertedIndexDescriptor::get_index_file_path_prefix(seg_path.value()); + auto inverted_index_file_reader = std::make_shared( + fs, std::string(index_file_path_prefix), + _tablet_schema->get_inverted_index_storage_format(), index_info); + EXPECT_TRUE(inverted_index_file_reader->init().ok()); + const auto& dirs = inverted_index_file_reader->get_all_directories(); + EXPECT_TRUE(dirs.has_value()); + EXPECT_EQ(dirs.value().size(), 4); + } + } + + CumulativeCompaction compaction(*_engine_ref, _tablet); + compaction._input_rowsets = std::move(rowsets); + compaction.build_basic_info(); + + std::vector input_rs_readers; + input_rs_readers.reserve(compaction._input_rowsets.size()); + for (auto& rowset : compaction._input_rowsets) { + RowsetReaderSharedPtr rs_reader; + EXPECT_TRUE(rowset->create_reader(&rs_reader).ok()); + input_rs_readers.push_back(std::move(rs_reader)); + } + + RowsetWriterContext ctx; + EXPECT_TRUE(compaction.construct_output_rowset_writer(ctx).ok()); + + // col word + EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(1)); + // col url + EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(2)); + + compaction._stats.rowid_conversion = compaction._rowid_conversion.get(); + EXPECT_TRUE(Merger::vertical_merge_rowsets(_tablet, compaction.compaction_type(), + *(compaction._cur_tablet_schema), input_rs_readers, + compaction._output_rs_writer.get(), 100000, 5, + &compaction._stats) + .ok()); + const auto& dst_writer = + dynamic_cast(compaction._output_rs_writer.get()); + for (const auto& [seg_id, idx_file_writer] : dst_writer->_idx_files.get_file_writers()) { + EXPECT_FALSE(idx_file_writer->_closed); + } + auto st = compaction.do_inverted_index_compaction(); + EXPECT_TRUE(st.ok()) << st.to_string(); + + st = compaction._output_rs_writer->build(compaction._output_rowset); + EXPECT_TRUE(st.ok()) << st.to_string(); + + for (const auto& [seg_id, idx_file_writer] : dst_writer->_idx_files.get_file_writers()) { + EXPECT_TRUE(idx_file_writer->_closed); + } + EXPECT_TRUE(compaction._output_rowset->num_segments() == 1); + + const auto& output_rowset = compaction._output_rowset; + + // check rowset meta and file + for (int seg_id = 0; seg_id < output_rowset->num_segments(); seg_id++) { + // meta + const auto& index_info = output_rowset->_rowset_meta->inverted_index_file_info(seg_id); + EXPECT_TRUE(index_info.has_index_size()); + const auto& fs = output_rowset->_rowset_meta->fs(); + const auto& file_name = fmt::format("{}/{}_{}.idx", output_rowset->tablet_path(), + output_rowset->rowset_id().to_string(), seg_id); + int64_t file_size = 0; + EXPECT_TRUE(fs->file_size(file_name, &file_size).ok()); + EXPECT_EQ(index_info.index_size(), file_size); + + // file + const auto& seg_path = output_rowset->segment_path(seg_id); + EXPECT_TRUE(seg_path.has_value()); + const auto& index_file_path_prefix = + InvertedIndexDescriptor::get_index_file_path_prefix(seg_path.value()); + auto inverted_index_file_reader = std::make_shared( + fs, std::string(index_file_path_prefix), + _tablet_schema->get_inverted_index_storage_format(), index_info); + EXPECT_TRUE(inverted_index_file_reader->init().ok()); + const auto& dirs = inverted_index_file_reader->get_all_directories(); + EXPECT_TRUE(dirs.has_value()); + EXPECT_EQ(dirs.value().size(), 4); + + // read col key + const auto& key = _tablet_schema->column_by_uid(0); + const auto* key_index = _tablet_schema->get_inverted_index(key); + EXPECT_TRUE(key_index != nullptr); + std::vector query_data {99, 66, 56, 87, 85, 96, 20000}; + std::vector query_result {21, 25, 22, 18, 14, 18, 0}; + EXPECT_TRUE(query_bkd(key_index, inverted_index_file_reader, query_data, query_result)); + + // read col v3 + const auto& v3_column = _tablet_schema->column_by_uid(3); + const auto* v3_index = _tablet_schema->get_inverted_index(v3_column); + EXPECT_TRUE(v3_index != nullptr); + std::vector query_data3 {99, 66, 56, 87, 85, 96, 10000}; + std::vector query_result3 {12, 20, 25, 23, 16, 24, 0}; + EXPECT_TRUE(query_bkd(v3_index, inverted_index_file_reader, query_data3, query_result3)); + + // read col v1 + const auto& v1_column = _tablet_schema->column_by_uid(1); + const auto* v1_index = _tablet_schema->get_inverted_index(v1_column); + EXPECT_TRUE(v1_index != nullptr); + std::vector query_data1 {"good", "maybe", "great", "null"}; + std::vector query_result1 {197, 191, 194, 0}; + EXPECT_TRUE(query_string(v1_index, inverted_index_file_reader, "1", query_data1, + query_result1)); + + // read col v2 + const auto& v2_column = _tablet_schema->column_by_uid(2); + const auto* v2_index = _tablet_schema->get_inverted_index(v2_column); + EXPECT_TRUE(v2_index != nullptr); + std::vector query_data2 {"musicstream.com", "http", "https", "null"}; + std::vector query_result2 {191, 799, 1201, 0}; + EXPECT_TRUE(query_fulltext(v2_index, inverted_index_file_reader, "2", query_data2, + query_result2)); + } +} + +} // namespace doris diff --git a/be/test/olap/rowset/segment_v2/inverted_index/index_compaction_test.cpp b/be/test/olap/rowset/segment_v2/inverted_index/compaction/index_compaction_write_index_test.cpp similarity index 96% rename from be/test/olap/rowset/segment_v2/inverted_index/index_compaction_test.cpp rename to be/test/olap/rowset/segment_v2/inverted_index/compaction/index_compaction_write_index_test.cpp index 1343cfd6e34612..ad24b174e7529f 100644 --- a/be/test/olap/rowset/segment_v2/inverted_index/index_compaction_test.cpp +++ b/be/test/olap/rowset/segment_v2/inverted_index/compaction/index_compaction_write_index_test.cpp @@ -51,7 +51,7 @@ CL_NS_USE(util) namespace doris::segment_v2 { -class IndexCompactionTest : public testing::Test { +class IndexCompactionWriteIndexTest : public testing::Test { public: const std::string kTestDir = "./ut_dir/index_compress_test"; @@ -65,8 +65,8 @@ class IndexCompactionTest : public testing::Test { EXPECT_TRUE(io::global_local_filesystem()->delete_directory(kTestDir).ok()); } - IndexCompactionTest() = default; - ~IndexCompactionTest() override = default; + IndexCompactionWriteIndexTest() = default; + ~IndexCompactionWriteIndexTest() override = default; static constexpr int32_t doc_count = 100000; }; @@ -167,7 +167,7 @@ static void index_compaction(const std::string& path, }) } -TEST_F(IndexCompactionTest, test_compaction_exception) { +TEST_F(IndexCompactionWriteIndexTest, test_compaction_exception) { std::srand(getDaySeed()); std::string name = "field_name"; diff --git a/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv b/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv new file mode 100644 index 00000000000000..fa4e2129584999 --- /dev/null +++ b/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv @@ -0,0 +1,1000 @@ +89,fine,https://musicstream.com,97 +44,good,https://yourblog.net,74 +72,ok,https://musicstream.com,80 +14,yes,https://shoponline.com,50 +47,maybe,https://github.com,16 +97,maybe,http://forum.com,88 +17,yes,https://musicstream.com,13 +87,good,https://musicstream.com,53 +6,excellent,http://forum.com,49 +4,terrible,https://yourblog.net,58 +56,bad,https://google.com,10 +5,no,http://example.com,93 +30,bad,http://news.com,28 +30,maybe,https://musicstream.com,13 +83,bad,https://yourblog.net,48 +41,great,https://videosite.com,73 +10,terrible,http://news.com,6 +80,no,https://shoponline.com,21 +92,fine,https://github.com,20 +71,terrible,http://forum.com,8 +84,good,http://mysite.org,77 +25,no,http://example.com,71 +52,maybe,http://mysite.org,0 +99,great,http://news.com,74 +59,ok,http://mysite.org,46 +47,ok,http://mysite.org,26 +77,bad,https://musicstream.com,73 +53,excellent,https://github.com,97 +97,ok,http://example.com,69 +88,great,https://musicstream.com,47 +31,fine,https://videosite.com,1 +66,great,https://google.com,67 +16,yes,http://forum.com,43 +31,yes,http://news.com,49 +35,good,https://github.com,48 +11,terrible,https://google.com,56 +53,maybe,http://mysite.org,87 +16,bad,https://google.com,77 +55,terrible,http://news.com,33 +51,yes,https://google.com,68 +71,good,https://yourblog.net,71 +32,great,https://github.com,37 +22,maybe,http://example.com,16 +36,ok,http://forum.com,52 +97,good,https://yourblog.net,100 +54,no,https://videosite.com,90 +38,terrible,https://shoponline.com,17 +81,excellent,https://yourblog.net,11 +66,great,http://mysite.org,4 +27,fine,https://shoponline.com,88 +48,good,https://musicstream.com,16 +78,good,https://shoponline.com,63 +34,bad,https://shoponline.com,77 +19,yes,https://shoponline.com,98 +39,terrible,https://github.com,86 +83,excellent,http://news.com,63 +38,no,https://google.com,64 +43,terrible,https://yourblog.net,30 +30,fine,https://videosite.com,72 +73,great,http://mysite.org,31 +96,good,http://example.com,16 +25,yes,http://mysite.org,80 +98,fine,https://shoponline.com,46 +60,fine,https://musicstream.com,66 +90,yes,http://news.com,80 +41,excellent,https://github.com,56 +86,no,http://forum.com,31 +81,ok,http://forum.com,71 +42,bad,https://musicstream.com,97 +90,terrible,https://videosite.com,57 +22,yes,http://forum.com,15 +34,excellent,https://shoponline.com,77 +30,no,https://yourblog.net,85 +51,ok,http://news.com,52 +28,terrible,http://example.com,37 +23,yes,http://forum.com,83 +42,no,http://forum.com,15 +83,maybe,http://forum.com,10 +59,ok,https://github.com,100 +83,maybe,http://mysite.org,78 +4,good,https://videosite.com,68 +63,good,http://forum.com,71 +61,ok,http://mysite.org,66 +83,great,http://example.com,75 +82,no,https://google.com,95 +71,terrible,https://musicstream.com,19 +42,bad,http://mysite.org,66 +14,great,https://google.com,83 +4,yes,https://yourblog.net,84 +49,great,https://github.com,2 +2,yes,http://mysite.org,84 +33,excellent,http://forum.com,78 +5,good,http://example.com,53 +5,great,https://yourblog.net,86 +80,yes,http://mysite.org,85 +8,great,https://shoponline.com,84 +26,no,https://github.com,56 +82,fine,https://shoponline.com,39 +0,bad,https://videosite.com,49 +30,terrible,https://videosite.com,41 +76,maybe,http://news.com,60 +44,maybe,http://example.com,33 +9,terrible,http://forum.com,93 +25,ok,https://yourblog.net,62 +45,great,https://google.com,17 +5,terrible,http://news.com,95 +49,good,http://example.com,97 +29,great,https://shoponline.com,69 +73,good,https://google.com,23 +94,yes,http://forum.com,25 +58,bad,https://videosite.com,32 +98,excellent,https://musicstream.com,36 +25,good,http://example.com,56 +14,ok,https://yourblog.net,34 +30,fine,http://example.com,38 +47,yes,http://mysite.org,71 +20,fine,http://news.com,1 +89,bad,https://google.com,50 +8,fine,https://google.com,64 +21,maybe,https://musicstream.com,11 +12,fine,https://videosite.com,48 +27,maybe,https://musicstream.com,2 +76,ok,https://musicstream.com,30 +38,excellent,https://videosite.com,54 +57,terrible,https://yourblog.net,22 +21,good,https://google.com,39 +32,good,https://yourblog.net,68 +47,terrible,https://google.com,49 +90,good,http://mysite.org,21 +5,yes,http://news.com,33 +49,no,http://news.com,54 +76,maybe,http://news.com,16 +24,maybe,http://forum.com,17 +77,good,http://example.com,92 +28,excellent,http://example.com,27 +27,no,https://github.com,52 +55,ok,http://forum.com,97 +85,bad,https://shoponline.com,43 +24,terrible,http://news.com,39 +30,excellent,http://mysite.org,72 +70,maybe,http://mysite.org,76 +14,good,https://videosite.com,32 +73,no,http://news.com,25 +29,terrible,http://mysite.org,45 +52,great,http://news.com,11 +26,yes,http://forum.com,17 +30,excellent,http://news.com,88 +95,great,https://musicstream.com,63 +33,great,https://github.com,70 +26,great,https://github.com,17 +76,great,https://shoponline.com,94 +93,bad,https://google.com,29 +48,no,https://yourblog.net,93 +42,yes,https://videosite.com,71 +4,fine,https://github.com,62 +34,maybe,http://forum.com,78 +36,yes,https://google.com,39 +48,bad,https://shoponline.com,17 +84,no,https://videosite.com,11 +87,ok,https://musicstream.com,51 +7,ok,https://google.com,73 +81,bad,https://github.com,91 +92,ok,https://google.com,82 +15,terrible,http://forum.com,64 +6,excellent,https://google.com,70 +35,bad,http://example.com,48 +96,maybe,https://shoponline.com,96 +25,fine,http://news.com,43 +9,no,http://mysite.org,14 +66,yes,https://videosite.com,87 +76,maybe,https://musicstream.com,13 +51,good,http://forum.com,1 +31,great,http://news.com,51 +72,fine,http://news.com,70 +63,no,http://forum.com,5 +73,great,https://yourblog.net,81 +13,terrible,https://github.com,27 +98,excellent,https://videosite.com,29 +71,good,http://news.com,68 +89,no,https://musicstream.com,60 +68,excellent,http://news.com,54 +76,good,https://google.com,79 +69,good,http://mysite.org,99 +35,good,http://news.com,70 +97,yes,https://yourblog.net,63 +44,ok,https://yourblog.net,4 +73,terrible,http://news.com,36 +37,yes,https://github.com,61 +26,fine,https://videosite.com,41 +37,excellent,https://musicstream.com,9 +18,yes,https://github.com,81 +54,excellent,https://shoponline.com,52 +73,great,https://yourblog.net,67 +19,bad,http://example.com,86 +1,terrible,http://mysite.org,16 +62,fine,https://musicstream.com,64 +61,excellent,https://shoponline.com,93 +38,bad,https://videosite.com,61 +55,great,http://news.com,35 +47,great,https://yourblog.net,42 +36,good,https://yourblog.net,40 +26,ok,http://example.com,58 +40,great,https://github.com,0 +81,fine,https://musicstream.com,92 +50,yes,https://github.com,21 +98,maybe,https://google.com,2 +37,bad,https://videosite.com,90 +39,excellent,https://yourblog.net,31 +81,ok,https://musicstream.com,90 +10,bad,http://news.com,18 +93,good,http://example.com,93 +65,fine,http://news.com,72 +10,yes,https://yourblog.net,77 +5,good,http://forum.com,52 +75,fine,https://github.com,67 +40,excellent,http://example.com,27 +21,good,http://example.com,0 +70,fine,http://forum.com,71 +72,no,https://videosite.com,87 +86,no,http://example.com,63 +5,yes,http://example.com,3 +5,fine,http://example.com,68 +36,excellent,https://videosite.com,77 +3,bad,https://github.com,12 +40,bad,http://example.com,53 +33,maybe,https://shoponline.com,3 +80,maybe,https://musicstream.com,49 +42,ok,https://musicstream.com,53 +17,ok,https://yourblog.net,54 +94,yes,https://musicstream.com,20 +16,great,https://videosite.com,42 +32,great,https://shoponline.com,96 +48,ok,http://mysite.org,34 +30,no,http://forum.com,99 +17,excellent,http://forum.com,57 +73,bad,http://forum.com,55 +24,no,https://videosite.com,97 +99,no,http://forum.com,96 +91,excellent,https://musicstream.com,85 +72,excellent,https://videosite.com,60 +74,maybe,http://mysite.org,1 +65,good,https://videosite.com,16 +58,maybe,http://forum.com,9 +97,good,http://example.com,68 +39,good,http://news.com,29 +30,no,http://forum.com,96 +99,maybe,http://mysite.org,64 +66,terrible,https://yourblog.net,0 +72,good,http://example.com,59 +85,terrible,https://yourblog.net,69 +41,great,https://github.com,97 +95,good,https://yourblog.net,42 +56,good,https://shoponline.com,43 +57,good,http://news.com,9 +68,good,http://example.com,80 +80,yes,https://videosite.com,14 +95,great,https://musicstream.com,43 +2,terrible,http://mysite.org,23 +94,terrible,https://shoponline.com,12 +45,maybe,https://github.com,67 +61,maybe,https://shoponline.com,46 +28,excellent,https://musicstream.com,10 +9,no,http://example.com,36 +75,great,https://google.com,43 +54,maybe,http://news.com,16 +49,bad,https://google.com,62 +33,fine,http://example.com,13 +93,excellent,http://example.com,11 +37,ok,https://videosite.com,67 +92,ok,http://forum.com,33 +1,excellent,https://shoponline.com,100 +87,no,http://mysite.org,0 +89,terrible,http://example.com,11 +26,yes,https://github.com,79 +44,terrible,https://yourblog.net,98 +69,excellent,http://news.com,71 +76,excellent,https://yourblog.net,26 +19,fine,https://google.com,16 +71,great,http://mysite.org,100 +99,no,http://forum.com,20 +98,fine,https://google.com,53 +48,fine,http://news.com,42 +29,maybe,http://news.com,29 +6,terrible,https://videosite.com,40 +77,great,http://forum.com,13 +34,terrible,https://yourblog.net,86 +46,terrible,https://musicstream.com,86 +80,great,https://videosite.com,31 +33,fine,https://google.com,80 +49,excellent,http://example.com,70 +80,ok,http://news.com,49 +72,yes,https://musicstream.com,78 +95,ok,https://videosite.com,4 +70,no,https://github.com,100 +1,yes,https://musicstream.com,35 +26,no,http://mysite.org,45 +72,good,https://videosite.com,28 +23,bad,http://mysite.org,56 +4,great,http://example.com,56 +15,great,https://github.com,52 +89,good,http://example.com,77 +85,ok,http://forum.com,86 +50,fine,http://mysite.org,7 +74,bad,https://videosite.com,78 +1,no,https://github.com,43 +71,maybe,https://github.com,11 +79,no,https://musicstream.com,57 +90,maybe,https://github.com,68 +19,bad,https://github.com,53 +53,good,https://google.com,59 +5,fine,http://mysite.org,67 +34,maybe,http://example.com,76 +40,terrible,https://google.com,50 +99,excellent,http://news.com,9 +77,bad,https://google.com,90 +36,bad,https://videosite.com,75 +85,excellent,https://github.com,40 +8,good,http://mysite.org,17 +43,good,http://mysite.org,79 +15,yes,https://musicstream.com,86 +22,maybe,https://yourblog.net,88 +86,yes,https://github.com,65 +18,terrible,http://news.com,48 +4,terrible,https://yourblog.net,85 +100,fine,https://yourblog.net,3 +41,good,http://forum.com,65 +64,terrible,https://github.com,34 +92,bad,https://google.com,21 +87,excellent,http://mysite.org,37 +30,excellent,https://github.com,14 +39,good,http://example.com,44 +68,yes,https://videosite.com,38 +31,maybe,https://yourblog.net,97 +36,bad,https://shoponline.com,20 +69,yes,https://shoponline.com,94 +45,good,http://mysite.org,30 +84,bad,http://news.com,73 +71,excellent,https://musicstream.com,26 +46,bad,http://mysite.org,4 +98,fine,https://yourblog.net,51 +17,excellent,https://videosite.com,77 +32,excellent,https://yourblog.net,23 +37,good,http://news.com,98 +60,ok,https://musicstream.com,24 +68,ok,https://videosite.com,44 +96,fine,https://musicstream.com,44 +65,yes,https://github.com,81 +25,terrible,https://musicstream.com,61 +54,terrible,https://shoponline.com,72 +5,fine,https://yourblog.net,93 +27,terrible,http://example.com,3 +30,bad,https://google.com,9 +99,excellent,http://forum.com,6 +31,yes,https://google.com,93 +82,good,https://google.com,56 +38,fine,https://google.com,56 +29,ok,https://yourblog.net,42 +91,no,https://google.com,62 +58,good,http://example.com,80 +75,fine,http://example.com,97 +59,maybe,https://google.com,13 +23,ok,http://mysite.org,38 +50,great,https://shoponline.com,43 +0,no,http://forum.com,98 +4,no,https://github.com,47 +20,fine,https://shoponline.com,52 +38,fine,http://example.com,21 +43,bad,http://mysite.org,15 +39,great,https://google.com,65 +92,yes,https://videosite.com,10 +35,terrible,http://mysite.org,53 +58,terrible,https://videosite.com,92 +56,yes,https://musicstream.com,49 +30,ok,https://shoponline.com,45 +24,excellent,https://github.com,68 +33,bad,http://forum.com,3 +87,no,http://forum.com,88 +31,fine,http://mysite.org,32 +40,yes,http://mysite.org,44 +17,yes,http://mysite.org,53 +4,fine,https://shoponline.com,31 +89,bad,https://musicstream.com,86 +24,excellent,https://github.com,97 +58,no,https://github.com,49 +26,ok,http://example.com,62 +59,great,https://shoponline.com,72 +22,yes,https://videosite.com,82 +27,great,http://example.com,24 +1,terrible,https://github.com,25 +2,bad,https://videosite.com,94 +39,good,http://news.com,16 +90,no,http://news.com,25 +6,no,https://yourblog.net,68 +73,ok,http://mysite.org,27 +68,no,https://google.com,64 +26,maybe,http://forum.com,63 +32,excellent,https://musicstream.com,58 +63,fine,https://videosite.com,35 +94,great,https://videosite.com,6 +8,ok,https://yourblog.net,16 +50,ok,https://videosite.com,46 +15,terrible,http://mysite.org,6 +53,excellent,https://github.com,30 +83,maybe,https://github.com,62 +78,good,https://shoponline.com,10 +96,fine,http://news.com,7 +73,ok,https://google.com,90 +95,great,https://shoponline.com,45 +61,excellent,https://github.com,26 +80,no,https://google.com,89 +80,great,http://forum.com,87 +12,excellent,https://musicstream.com,35 +59,great,https://musicstream.com,60 +27,ok,http://example.com,94 +68,maybe,https://google.com,25 +19,terrible,https://yourblog.net,77 +59,yes,http://mysite.org,67 +65,great,https://musicstream.com,70 +38,good,https://google.com,63 +2,no,http://example.com,40 +7,great,http://forum.com,32 +93,great,https://yourblog.net,47 +66,excellent,http://mysite.org,89 +28,excellent,https://shoponline.com,42 +28,no,https://google.com,71 +83,terrible,https://videosite.com,82 +33,good,https://musicstream.com,87 +24,terrible,http://news.com,30 +87,yes,http://example.com,25 +0,ok,http://example.com,87 +93,bad,https://musicstream.com,24 +16,yes,http://mysite.org,63 +81,yes,http://example.com,0 +85,yes,https://videosite.com,96 +62,bad,http://mysite.org,86 +43,great,https://shoponline.com,56 +62,no,https://musicstream.com,45 +35,terrible,http://forum.com,66 +65,ok,https://videosite.com,95 +70,excellent,https://yourblog.net,13 +50,bad,https://google.com,94 +54,no,http://forum.com,44 +1,bad,https://videosite.com,4 +82,yes,https://yourblog.net,9 +96,maybe,https://videosite.com,3 +87,excellent,https://musicstream.com,14 +99,fine,https://google.com,36 +38,bad,http://example.com,71 +21,terrible,http://mysite.org,91 +23,ok,https://yourblog.net,17 +88,good,http://news.com,23 +83,great,https://musicstream.com,68 +25,good,http://news.com,93 +60,fine,https://google.com,89 +45,excellent,http://example.com,21 +59,bad,https://google.com,38 +22,bad,https://shoponline.com,11 +54,maybe,http://example.com,97 +18,great,http://forum.com,35 +37,ok,https://google.com,11 +59,no,http://news.com,68 +5,great,http://example.com,12 +32,yes,https://yourblog.net,15 +91,fine,http://forum.com,12 +40,no,http://news.com,40 +87,terrible,https://videosite.com,28 +15,no,https://videosite.com,52 +63,yes,http://forum.com,39 +81,terrible,http://news.com,36 +53,good,https://google.com,45 +12,excellent,https://shoponline.com,42 +20,ok,http://news.com,78 +65,excellent,http://forum.com,65 +64,maybe,https://shoponline.com,100 +73,no,https://yourblog.net,86 +44,great,http://example.com,58 +7,good,https://yourblog.net,21 +10,good,https://google.com,33 +11,no,https://musicstream.com,24 +59,no,https://github.com,12 +35,yes,http://news.com,65 +78,good,http://example.com,97 +22,no,https://google.com,2 +64,excellent,http://mysite.org,17 +18,ok,https://shoponline.com,21 +91,fine,https://musicstream.com,50 +92,great,http://news.com,9 +13,terrible,https://github.com,17 +13,excellent,https://yourblog.net,43 +93,no,http://mysite.org,80 +90,fine,http://forum.com,59 +33,great,https://videosite.com,54 +94,bad,http://news.com,87 +52,maybe,https://videosite.com,63 +55,terrible,http://forum.com,4 +12,yes,https://videosite.com,5 +50,terrible,http://forum.com,35 +42,bad,https://google.com,77 +69,great,https://shoponline.com,57 +74,maybe,https://google.com,66 +52,ok,http://example.com,86 +18,yes,https://google.com,7 +7,fine,http://forum.com,94 +11,great,https://google.com,69 +76,excellent,https://shoponline.com,79 +20,excellent,https://yourblog.net,98 +60,no,https://videosite.com,21 +52,no,http://forum.com,33 +52,excellent,http://forum.com,57 +48,ok,https://github.com,33 +32,excellent,https://musicstream.com,74 +39,ok,https://yourblog.net,88 +98,excellent,http://example.com,0 +3,excellent,http://example.com,97 +76,excellent,http://mysite.org,16 +44,good,https://videosite.com,45 +51,great,http://mysite.org,96 +44,ok,http://news.com,62 +26,excellent,http://news.com,54 +69,fine,https://shoponline.com,12 +16,excellent,http://forum.com,24 +36,bad,https://musicstream.com,54 +12,good,http://news.com,58 +72,bad,https://videosite.com,87 +2,yes,http://mysite.org,62 +38,yes,https://shoponline.com,29 +31,good,https://yourblog.net,36 +43,bad,https://google.com,34 +72,fine,https://shoponline.com,96 +8,fine,https://yourblog.net,94 +29,ok,https://github.com,90 +53,ok,http://mysite.org,12 +12,excellent,http://forum.com,41 +25,terrible,http://mysite.org,19 +97,bad,https://yourblog.net,32 +2,good,http://mysite.org,3 +96,bad,https://github.com,35 +67,yes,http://forum.com,15 +37,fine,https://google.com,74 +57,no,https://google.com,62 +50,excellent,http://forum.com,21 +40,terrible,https://google.com,29 +68,good,https://shoponline.com,63 +51,maybe,http://news.com,35 +0,great,http://example.com,96 +71,great,http://mysite.org,46 +33,yes,http://news.com,23 +51,great,https://videosite.com,21 +16,excellent,http://example.com,69 +88,no,https://github.com,73 +7,maybe,https://github.com,76 +33,terrible,https://musicstream.com,39 +92,great,https://yourblog.net,47 +57,maybe,https://videosite.com,32 +37,ok,https://videosite.com,20 +12,maybe,https://musicstream.com,84 +72,bad,https://google.com,44 +35,ok,http://mysite.org,4 +17,yes,http://mysite.org,22 +62,excellent,http://news.com,35 +33,bad,https://yourblog.net,66 +45,excellent,https://shoponline.com,38 +99,fine,https://yourblog.net,93 +83,terrible,https://videosite.com,55 +61,fine,https://videosite.com,56 +91,yes,http://forum.com,72 +91,yes,http://news.com,15 +89,fine,http://example.com,17 +32,ok,http://example.com,86 +50,bad,http://mysite.org,65 +12,fine,https://musicstream.com,30 +45,no,http://news.com,75 +42,terrible,https://shoponline.com,95 +48,excellent,http://mysite.org,50 +30,terrible,https://yourblog.net,34 +54,good,http://forum.com,58 +2,excellent,http://mysite.org,89 +15,no,http://news.com,61 +88,fine,https://google.com,76 +96,no,https://musicstream.com,26 +98,fine,https://yourblog.net,18 +52,maybe,https://videosite.com,17 +34,great,https://yourblog.net,20 +80,ok,https://shoponline.com,70 +13,ok,http://news.com,17 +2,terrible,http://forum.com,3 +86,good,http://news.com,68 +51,bad,http://example.com,7 +29,maybe,https://videosite.com,85 +5,no,http://mysite.org,53 +90,ok,http://forum.com,33 +80,fine,https://google.com,62 +52,yes,http://forum.com,2 +38,yes,http://news.com,90 +96,ok,https://videosite.com,9 +13,ok,https://github.com,98 +18,maybe,http://example.com,59 +22,yes,https://videosite.com,40 +36,great,http://example.com,24 +66,terrible,https://shoponline.com,46 +99,maybe,http://example.com,92 +85,good,http://forum.com,96 +14,fine,https://google.com,68 +7,good,http://news.com,75 +37,no,https://shoponline.com,47 +86,excellent,https://yourblog.net,100 +70,no,http://mysite.org,7 +40,yes,http://mysite.org,36 +92,good,https://shoponline.com,92 +79,excellent,http://example.com,80 +58,no,https://yourblog.net,99 +28,yes,https://yourblog.net,36 +81,great,http://mysite.org,36 +14,excellent,https://musicstream.com,23 +18,no,https://musicstream.com,54 +17,great,https://musicstream.com,3 +21,great,https://shoponline.com,15 +10,terrible,https://musicstream.com,68 +94,terrible,http://example.com,13 +2,good,https://google.com,55 +76,yes,https://musicstream.com,84 +10,maybe,https://google.com,99 +97,yes,http://forum.com,42 +63,excellent,https://videosite.com,11 +13,ok,http://example.com,87 +76,great,http://news.com,40 +75,good,https://yourblog.net,98 +35,great,https://shoponline.com,94 +71,maybe,http://mysite.org,6 +34,no,http://forum.com,59 +29,fine,http://example.com,4 +1,yes,http://mysite.org,27 +74,maybe,https://shoponline.com,14 +26,great,http://news.com,84 +30,no,https://videosite.com,1 +21,excellent,https://google.com,81 +74,yes,http://example.com,24 +23,no,https://videosite.com,88 +46,good,https://musicstream.com,77 +62,ok,https://shoponline.com,97 +37,ok,https://yourblog.net,36 +48,excellent,https://yourblog.net,4 +27,excellent,https://videosite.com,33 +24,terrible,https://google.com,17 +1,yes,http://news.com,28 +57,ok,http://example.com,93 +84,good,https://yourblog.net,88 +92,great,https://musicstream.com,61 +64,yes,https://musicstream.com,85 +94,ok,https://yourblog.net,96 +67,ok,http://forum.com,29 +66,great,https://github.com,67 +91,bad,http://example.com,19 +95,great,http://mysite.org,0 +68,ok,http://example.com,67 +3,excellent,https://github.com,49 +61,maybe,https://yourblog.net,100 +19,bad,https://videosite.com,100 +58,excellent,http://forum.com,25 +45,ok,https://yourblog.net,57 +43,maybe,https://shoponline.com,52 +60,terrible,http://example.com,97 +19,bad,https://yourblog.net,76 +26,fine,https://musicstream.com,76 +51,excellent,https://videosite.com,88 +84,fine,http://mysite.org,31 +99,great,https://github.com,30 +60,terrible,http://forum.com,48 +28,ok,https://yourblog.net,58 +23,terrible,https://musicstream.com,37 +41,great,https://shoponline.com,1 +18,ok,https://github.com,6 +39,ok,http://mysite.org,65 +14,terrible,https://google.com,96 +86,fine,http://forum.com,15 +24,yes,https://yourblog.net,1 +86,good,https://github.com,9 +20,good,http://news.com,3 +94,maybe,https://github.com,52 +19,yes,http://forum.com,88 +0,great,https://google.com,37 +5,ok,https://yourblog.net,69 +29,terrible,https://google.com,56 +26,great,https://musicstream.com,89 +64,fine,https://yourblog.net,5 +51,yes,https://yourblog.net,15 +14,excellent,https://google.com,89 +48,ok,https://github.com,60 +77,great,https://musicstream.com,80 +28,bad,https://musicstream.com,46 +8,no,https://videosite.com,32 +48,great,https://google.com,9 +91,ok,https://github.com,12 +82,great,https://videosite.com,2 +99,terrible,http://forum.com,46 +82,maybe,http://forum.com,21 +73,good,http://forum.com,51 +32,bad,https://google.com,30 +52,maybe,http://mysite.org,72 +53,maybe,https://github.com,100 +100,bad,http://example.com,53 +8,ok,http://example.com,28 +62,bad,http://example.com,56 +61,yes,https://shoponline.com,17 +44,bad,http://mysite.org,32 +54,good,https://shoponline.com,16 +4,bad,http://forum.com,75 +65,excellent,http://news.com,4 +72,yes,https://google.com,25 +68,ok,https://shoponline.com,72 +81,excellent,https://yourblog.net,19 +99,good,https://shoponline.com,9 +21,ok,http://news.com,86 +14,maybe,https://videosite.com,83 +63,bad,http://example.com,20 +14,bad,http://example.com,66 +9,no,https://shoponline.com,61 +53,fine,https://shoponline.com,6 +6,no,https://github.com,86 +69,good,https://videosite.com,23 +59,fine,https://musicstream.com,30 +35,maybe,https://github.com,44 +87,fine,https://musicstream.com,17 +55,excellent,https://musicstream.com,82 +22,fine,https://yourblog.net,88 +65,no,https://musicstream.com,22 +90,great,https://google.com,14 +49,good,https://github.com,77 +84,great,https://musicstream.com,10 +31,great,https://github.com,82 +48,terrible,http://news.com,74 +12,fine,http://mysite.org,74 +13,great,http://forum.com,28 +92,no,https://videosite.com,81 +6,no,https://videosite.com,100 +0,good,http://news.com,73 +33,good,https://google.com,38 +51,terrible,https://shoponline.com,17 +72,terrible,https://yourblog.net,52 +47,bad,https://shoponline.com,59 +41,ok,https://yourblog.net,98 +68,great,https://videosite.com,94 +77,good,http://news.com,87 +39,ok,http://example.com,89 +47,excellent,http://example.com,56 +31,bad,http://mysite.org,56 +53,fine,https://shoponline.com,18 +8,yes,https://github.com,63 +99,yes,http://example.com,70 +7,ok,http://example.com,85 +35,bad,https://shoponline.com,14 +73,terrible,https://yourblog.net,84 +26,no,https://musicstream.com,84 +43,bad,https://videosite.com,44 +10,great,https://google.com,48 +53,terrible,https://github.com,55 +58,good,http://forum.com,14 +27,fine,http://news.com,67 +73,ok,http://example.com,9 +28,ok,https://google.com,84 +78,maybe,https://yourblog.net,59 +79,great,https://shoponline.com,64 +16,ok,https://yourblog.net,67 +75,yes,https://musicstream.com,3 +33,good,https://shoponline.com,12 +78,great,https://yourblog.net,30 +90,yes,https://videosite.com,17 +29,ok,http://example.com,40 +94,good,http://example.com,88 +30,maybe,https://videosite.com,93 +57,no,https://google.com,35 +50,no,https://yourblog.net,55 +65,maybe,https://videosite.com,47 +85,maybe,https://google.com,74 +5,maybe,https://yourblog.net,4 +16,terrible,https://google.com,50 +69,yes,https://musicstream.com,45 +47,terrible,http://example.com,4 +27,ok,http://example.com,60 +17,yes,http://example.com,43 +8,yes,https://musicstream.com,98 +75,no,https://github.com,78 +41,great,https://google.com,34 +88,bad,http://example.com,78 +94,good,http://news.com,41 +26,fine,http://forum.com,37 +23,excellent,https://shoponline.com,72 +76,fine,https://yourblog.net,97 +72,terrible,https://shoponline.com,56 +57,fine,https://videosite.com,80 +42,excellent,https://musicstream.com,40 +96,fine,https://shoponline.com,69 +13,fine,https://yourblog.net,22 +2,fine,https://shoponline.com,43 +39,excellent,https://yourblog.net,41 +18,excellent,http://example.com,55 +14,no,https://google.com,52 +30,no,https://yourblog.net,44 +42,good,https://videosite.com,37 +98,yes,https://videosite.com,44 +77,ok,https://musicstream.com,48 +39,ok,https://yourblog.net,41 +60,yes,http://news.com,41 +30,good,https://musicstream.com,61 +33,fine,https://github.com,98 +25,fine,https://videosite.com,45 +43,yes,http://mysite.org,29 +69,maybe,https://musicstream.com,9 +56,bad,https://github.com,83 +29,ok,https://yourblog.net,43 +57,no,https://musicstream.com,99 +72,bad,http://mysite.org,33 +0,terrible,https://videosite.com,70 +43,ok,https://shoponline.com,79 +97,terrible,http://example.com,2 +45,bad,https://musicstream.com,98 +52,no,https://google.com,11 +69,bad,https://videosite.com,14 +52,yes,https://google.com,17 +89,fine,https://github.com,36 +6,yes,https://videosite.com,67 +72,excellent,http://mysite.org,68 +0,maybe,https://google.com,24 +95,yes,https://shoponline.com,62 +40,maybe,https://yourblog.net,14 +41,no,https://videosite.com,87 +58,fine,https://yourblog.net,43 +27,no,http://news.com,49 +46,great,http://example.com,1 +91,maybe,http://news.com,58 +34,no,https://github.com,19 +63,no,http://example.com,71 +100,good,https://musicstream.com,94 +81,yes,https://shoponline.com,46 +95,great,http://forum.com,5 +84,yes,https://shoponline.com,45 +72,excellent,https://github.com,73 +40,no,https://videosite.com,6 +6,excellent,http://news.com,10 +62,great,https://musicstream.com,82 +76,terrible,http://forum.com,4 +13,no,http://mysite.org,86 +36,great,http://example.com,83 +67,maybe,https://yourblog.net,43 +22,excellent,http://example.com,35 +18,great,https://google.com,15 +65,terrible,https://github.com,30 +44,fine,http://forum.com,18 +3,excellent,http://forum.com,37 +10,bad,http://forum.com,65 +17,ok,https://shoponline.com,66 +3,maybe,https://shoponline.com,13 +51,maybe,https://yourblog.net,28 +42,bad,http://news.com,23 +70,excellent,https://yourblog.net,91 +94,bad,https://videosite.com,20 +5,terrible,http://example.com,85 +21,excellent,https://yourblog.net,66 +71,fine,https://yourblog.net,59 +33,terrible,http://news.com,16 +36,bad,https://shoponline.com,86 +45,terrible,https://github.com,94 +62,excellent,https://github.com,42 +80,terrible,http://mysite.org,49 +81,bad,http://news.com,41 +68,bad,http://forum.com,54 +80,bad,https://shoponline.com,15 +63,good,https://videosite.com,34 +67,ok,http://example.com,59 +40,terrible,https://google.com,55 +34,bad,https://yourblog.net,49 +58,no,https://yourblog.net,37 +70,terrible,https://yourblog.net,19 +90,great,http://example.com,78 +5,maybe,https://github.com,96 +34,bad,http://example.com,100 +21,ok,http://news.com,32 +0,no,https://musicstream.com,92 +44,yes,https://musicstream.com,27 +71,bad,https://videosite.com,79 +6,terrible,http://mysite.org,46 +72,excellent,https://musicstream.com,50 +20,maybe,https://google.com,37 +56,great,http://mysite.org,22 +35,maybe,https://shoponline.com,18 +40,terrible,http://mysite.org,33 +40,yes,http://forum.com,39 +95,excellent,https://google.com,58 +31,great,https://musicstream.com,40 +45,good,http://forum.com,46 +59,excellent,http://example.com,44 +11,no,https://videosite.com,33 +78,terrible,https://google.com,92 +70,terrible,https://videosite.com,18 +38,excellent,https://videosite.com,97 +21,yes,http://forum.com,22 +30,fine,http://example.com,46 +64,fine,https://github.com,83 +22,no,https://shoponline.com,92 +14,good,http://forum.com,90 +93,maybe,https://google.com,22 +68,no,https://videosite.com,79 +13,maybe,https://videosite.com,47 +98,great,http://news.com,16 +3,great,https://videosite.com,72 +90,good,https://yourblog.net,78 +85,terrible,http://forum.com,92 +1,fine,https://videosite.com,74 +90,yes,http://forum.com,90 +86,yes,http://example.com,31 +40,terrible,https://github.com,84 +71,yes,https://shoponline.com,69 +4,terrible,https://shoponline.com,15 +74,excellent,http://news.com,97 +97,great,https://github.com,79 +67,great,https://videosite.com,76 +94,bad,https://github.com,51 +26,good,https://github.com,33 +52,fine,https://musicstream.com,17 +43,yes,https://github.com,12 +71,excellent,https://shoponline.com,53 +51,no,http://mysite.org,75 +46,excellent,https://shoponline.com,84 +84,excellent,https://videosite.com,79 +33,maybe,http://mysite.org,45 +17,yes,https://yourblog.net,71 +0,maybe,https://shoponline.com,32 +77,no,https://google.com,24 +5,fine,http://news.com,51 +20,great,https://shoponline.com,58 +26,fine,https://google.com,61 +11,terrible,http://forum.com,85 +14,fine,https://github.com,84 +54,ok,https://google.com,60 +7,excellent,https://google.com,47 +53,no,https://shoponline.com,38 +56,good,https://yourblog.net,35 +60,bad,https://shoponline.com,63 +62,great,https://videosite.com,21 +58,maybe,https://yourblog.net,19 +59,good,https://yourblog.net,11 +40,bad,https://github.com,61 +4,yes,http://example.com,81 +70,great,https://yourblog.net,6 +53,great,https://yourblog.net,37 +50,no,http://example.com,72 +97,ok,https://google.com,16 +83,fine,https://yourblog.net,84 +44,maybe,http://forum.com,38 +77,yes,https://github.com,55 +1,no,https://musicstream.com,22 +35,no,https://google.com,99 +88,excellent,https://yourblog.net,45 +62,ok,http://news.com,16 +51,fine,http://news.com,93 +17,fine,https://musicstream.com,44 +44,maybe,https://videosite.com,32 +6,terrible,https://videosite.com,39 +79,terrible,http://example.com,89 +41,great,https://shoponline.com,95 +63,yes,http://forum.com,92 +34,yes,http://example.com,28 +35,excellent,https://musicstream.com,25 +95,bad,https://shoponline.com,22 +47,maybe,http://example.com,50 +5,terrible,https://musicstream.com,6 +37,fine,http://forum.com,92 +94,good,https://github.com,33 +14,no,http://example.com,95 +63,excellent,http://forum.com,36 +24,fine,https://videosite.com,98 +7,fine,https://yourblog.net,71 +48,good,http://mysite.org,34 +32,fine,https://yourblog.net,31 +48,great,http://news.com,14 +35,no,http://example.com,77 +72,fine,https://google.com,37 +55,excellent,https://google.com,46 +63,great,http://mysite.org,60 +1,excellent,https://github.com,44 +37,fine,https://shoponline.com,72 +97,no,https://yourblog.net,88 +76,excellent,http://mysite.org,11 +88,ok,https://github.com,92 +83,no,http://news.com,44 +44,fine,https://videosite.com,64 +66,fine,https://google.com,87 +31,ok,http://mysite.org,64 +20,good,https://yourblog.net,69 +68,fine,http://forum.com,63 +41,terrible,http://example.com,47 +81,maybe,https://shoponline.com,33 +60,maybe,http://mysite.org,59 +26,good,http://news.com,87 +1,excellent,https://google.com,84 +79,ok,http://example.com,56 +72,excellent,http://mysite.org,64 +0,good,https://videosite.com,1 +97,no,http://news.com,91 +34,terrible,http://forum.com,36 diff --git a/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv b/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv new file mode 100644 index 00000000000000..0c8a15bafdd575 --- /dev/null +++ b/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv @@ -0,0 +1,1000 @@ +37,no,http://example.com,34 +59,ok,https://shoponline.com,9 +44,ok,http://news.com,40 +64,no,https://github.com,37 +61,no,https://musicstream.com,42 +68,great,http://mysite.org,49 +23,good,http://example.com,90 +49,fine,https://yourblog.net,23 +34,excellent,https://musicstream.com,69 +23,maybe,https://yourblog.net,15 +19,great,http://example.com,61 +68,excellent,https://github.com,78 +65,yes,https://shoponline.com,67 +84,excellent,https://videosite.com,8 +45,terrible,http://news.com,96 +13,bad,https://videosite.com,100 +8,great,http://mysite.org,66 +15,fine,http://news.com,18 +35,bad,https://videosite.com,27 +42,bad,https://videosite.com,0 +100,fine,http://forum.com,89 +94,yes,https://shoponline.com,28 +26,good,https://google.com,6 +55,yes,https://google.com,0 +66,excellent,https://google.com,30 +78,terrible,https://github.com,57 +31,excellent,http://news.com,61 +86,fine,http://mysite.org,30 +8,maybe,https://shoponline.com,15 +43,yes,http://mysite.org,57 +79,bad,http://forum.com,26 +28,no,http://mysite.org,64 +29,good,http://mysite.org,25 +17,excellent,https://shoponline.com,87 +72,good,http://forum.com,33 +89,excellent,https://google.com,100 +7,bad,https://google.com,22 +33,excellent,https://github.com,23 +42,fine,https://shoponline.com,79 +96,fine,https://github.com,94 +91,maybe,http://news.com,61 +53,good,http://forum.com,12 +96,yes,http://news.com,33 +20,bad,https://shoponline.com,9 +86,fine,https://musicstream.com,48 +76,maybe,https://google.com,38 +33,yes,https://videosite.com,35 +73,ok,https://videosite.com,40 +41,no,http://news.com,96 +15,bad,http://example.com,44 +18,excellent,https://shoponline.com,11 +46,excellent,https://yourblog.net,71 +87,bad,http://news.com,37 +50,yes,http://news.com,94 +80,excellent,https://musicstream.com,91 +95,maybe,https://google.com,45 +48,terrible,https://github.com,58 +91,ok,http://news.com,90 +42,yes,http://mysite.org,45 +27,excellent,https://github.com,50 +11,ok,https://github.com,61 +61,excellent,https://shoponline.com,54 +1,bad,http://mysite.org,20 +9,yes,http://example.com,30 +81,good,http://news.com,47 +24,ok,https://yourblog.net,52 +1,great,https://yourblog.net,34 +78,ok,http://mysite.org,51 +45,fine,http://news.com,46 +46,excellent,http://mysite.org,63 +18,bad,https://musicstream.com,9 +88,fine,https://github.com,65 +45,terrible,https://videosite.com,36 +79,no,http://forum.com,47 +59,great,https://videosite.com,20 +9,good,http://example.com,38 +17,maybe,https://yourblog.net,55 +52,terrible,http://forum.com,99 +32,maybe,https://videosite.com,12 +81,great,http://example.com,10 +27,yes,https://shoponline.com,5 +29,maybe,https://shoponline.com,32 +80,maybe,https://musicstream.com,4 +15,yes,http://forum.com,26 +58,maybe,https://github.com,39 +13,no,https://shoponline.com,13 +76,great,http://example.com,97 +15,excellent,https://yourblog.net,6 +34,good,http://mysite.org,57 +79,no,https://musicstream.com,85 +49,terrible,https://github.com,90 +92,great,https://google.com,60 +41,great,https://shoponline.com,27 +54,good,http://news.com,2 +24,maybe,http://news.com,57 +3,fine,https://musicstream.com,32 +42,bad,https://shoponline.com,48 +67,good,http://mysite.org,67 +86,ok,https://yourblog.net,6 +19,ok,https://google.com,23 +60,great,http://forum.com,42 +20,good,https://musicstream.com,21 +96,no,http://news.com,94 +4,good,https://videosite.com,85 +4,no,https://github.com,87 +28,bad,https://google.com,21 +85,excellent,https://musicstream.com,78 +80,great,http://mysite.org,18 +22,fine,https://videosite.com,19 +76,maybe,https://github.com,5 +55,bad,https://yourblog.net,49 +50,ok,https://shoponline.com,79 +86,terrible,https://google.com,84 +52,excellent,https://yourblog.net,72 +23,ok,https://musicstream.com,30 +15,excellent,https://videosite.com,23 +1,terrible,https://google.com,23 +59,terrible,http://mysite.org,71 +66,fine,https://musicstream.com,49 +37,bad,http://forum.com,18 +35,ok,https://videosite.com,53 +83,bad,https://google.com,90 +54,excellent,http://news.com,7 +52,fine,http://example.com,71 +51,fine,https://musicstream.com,5 +88,good,http://example.com,5 +0,terrible,https://yourblog.net,91 +72,great,https://videosite.com,34 +8,no,http://forum.com,76 +97,bad,https://yourblog.net,20 +40,fine,https://yourblog.net,76 +29,terrible,http://mysite.org,29 +5,yes,https://shoponline.com,6 +66,maybe,https://musicstream.com,97 +65,fine,https://yourblog.net,70 +3,terrible,http://mysite.org,96 +56,excellent,http://example.com,7 +93,yes,https://google.com,33 +11,terrible,https://github.com,87 +10,yes,http://forum.com,9 +28,terrible,http://mysite.org,40 +85,bad,https://google.com,71 +47,maybe,https://yourblog.net,45 +56,maybe,https://videosite.com,73 +30,terrible,http://mysite.org,22 +66,maybe,http://mysite.org,17 +34,good,https://google.com,56 +0,excellent,http://example.com,54 +24,yes,https://github.com,94 +50,good,https://videosite.com,70 +39,good,http://example.com,62 +41,terrible,http://forum.com,50 +26,great,http://example.com,79 +100,yes,https://google.com,30 +37,great,https://shoponline.com,45 +25,terrible,http://forum.com,72 +82,bad,https://yourblog.net,66 +62,ok,http://news.com,5 +67,excellent,https://musicstream.com,65 +26,maybe,http://mysite.org,37 +77,yes,http://forum.com,100 +32,terrible,https://musicstream.com,49 +20,great,https://yourblog.net,5 +45,maybe,https://google.com,30 +76,good,http://mysite.org,16 +79,great,https://github.com,91 +40,good,https://yourblog.net,83 +89,no,https://yourblog.net,92 +45,yes,http://mysite.org,23 +47,great,http://mysite.org,41 +80,terrible,https://musicstream.com,24 +49,good,https://shoponline.com,100 +85,terrible,https://yourblog.net,3 +31,great,https://github.com,53 +85,fine,https://videosite.com,53 +85,terrible,http://mysite.org,100 +42,bad,https://github.com,73 +93,no,http://forum.com,32 +64,excellent,http://example.com,92 +79,terrible,https://videosite.com,83 +83,bad,http://mysite.org,15 +41,yes,https://shoponline.com,73 +85,yes,http://forum.com,63 +43,yes,http://news.com,23 +27,bad,https://musicstream.com,57 +87,yes,https://google.com,82 +22,fine,https://musicstream.com,0 +68,good,https://shoponline.com,92 +79,excellent,https://videosite.com,39 +68,good,http://forum.com,18 +77,yes,https://github.com,13 +38,great,https://google.com,56 +91,maybe,https://yourblog.net,7 +59,bad,http://news.com,69 +24,great,http://forum.com,53 +91,great,http://news.com,12 +59,bad,https://musicstream.com,15 +14,excellent,https://musicstream.com,40 +79,maybe,http://mysite.org,96 +4,yes,http://forum.com,4 +15,bad,http://news.com,34 +87,good,http://news.com,79 +21,bad,https://github.com,21 +62,terrible,https://google.com,34 +54,ok,https://google.com,24 +57,no,https://videosite.com,9 +40,good,https://shoponline.com,60 +97,maybe,https://google.com,38 +99,no,https://yourblog.net,64 +24,no,https://shoponline.com,82 +17,fine,http://example.com,46 +55,no,https://google.com,47 +47,fine,http://forum.com,54 +17,yes,https://google.com,44 +28,ok,http://news.com,46 +57,excellent,https://google.com,79 +70,yes,https://google.com,36 +46,excellent,https://shoponline.com,71 +30,terrible,https://videosite.com,16 +55,excellent,https://shoponline.com,69 +90,maybe,https://videosite.com,73 +57,terrible,https://videosite.com,20 +7,ok,http://news.com,23 +53,yes,http://news.com,73 +16,bad,https://github.com,56 +35,ok,https://musicstream.com,97 +82,excellent,https://videosite.com,66 +22,yes,https://videosite.com,59 +70,no,http://mysite.org,57 +32,yes,https://shoponline.com,39 +78,yes,https://github.com,33 +70,ok,http://news.com,0 +38,fine,https://shoponline.com,58 +93,no,https://shoponline.com,34 +51,excellent,https://musicstream.com,11 +77,bad,http://example.com,52 +78,ok,http://forum.com,65 +30,bad,https://videosite.com,20 +41,ok,https://github.com,9 +68,yes,https://videosite.com,17 +88,terrible,http://news.com,99 +15,terrible,https://github.com,55 +38,ok,http://forum.com,78 +8,maybe,http://mysite.org,67 +77,bad,http://forum.com,46 +98,terrible,http://news.com,93 +42,maybe,https://videosite.com,57 +23,yes,https://musicstream.com,51 +79,great,http://mysite.org,27 +83,fine,http://example.com,8 +4,yes,http://news.com,48 +99,yes,http://forum.com,51 +40,no,https://google.com,59 +5,terrible,http://example.com,80 +33,great,https://yourblog.net,4 +36,maybe,https://videosite.com,36 +66,maybe,https://shoponline.com,30 +46,maybe,https://musicstream.com,30 +36,great,http://news.com,3 +79,no,http://forum.com,31 +77,excellent,https://google.com,30 +66,terrible,https://google.com,19 +12,yes,https://google.com,28 +13,terrible,https://yourblog.net,40 +46,yes,http://example.com,59 +91,terrible,https://videosite.com,84 +99,maybe,https://videosite.com,29 +100,fine,https://yourblog.net,81 +35,no,http://mysite.org,19 +11,no,https://google.com,28 +71,terrible,http://forum.com,98 +91,great,https://musicstream.com,87 +89,bad,https://videosite.com,28 +79,yes,https://musicstream.com,88 +48,fine,https://yourblog.net,4 +26,ok,https://yourblog.net,66 +77,great,https://shoponline.com,8 +80,fine,https://musicstream.com,37 +63,fine,http://forum.com,100 +95,excellent,http://news.com,90 +65,great,http://mysite.org,71 +21,yes,http://mysite.org,76 +84,terrible,https://shoponline.com,100 +38,ok,https://videosite.com,80 +17,maybe,https://shoponline.com,62 +51,bad,http://mysite.org,18 +71,excellent,https://github.com,60 +77,yes,https://musicstream.com,18 +1,bad,https://musicstream.com,55 +48,terrible,https://google.com,80 +15,bad,https://shoponline.com,68 +68,ok,https://videosite.com,7 +54,bad,https://musicstream.com,72 +79,excellent,https://videosite.com,53 +69,maybe,http://mysite.org,52 +66,bad,https://github.com,18 +86,no,https://shoponline.com,27 +30,great,https://google.com,90 +44,maybe,http://forum.com,80 +90,maybe,https://google.com,49 +84,good,https://github.com,18 +51,good,http://forum.com,69 +68,bad,http://mysite.org,17 +50,good,https://github.com,8 +39,great,http://news.com,45 +53,bad,https://shoponline.com,75 +45,yes,https://github.com,59 +70,bad,http://news.com,4 +83,ok,http://forum.com,98 +33,great,https://shoponline.com,48 +83,bad,https://yourblog.net,97 +34,yes,http://example.com,41 +78,yes,https://shoponline.com,70 +49,yes,http://mysite.org,9 +59,maybe,http://mysite.org,88 +75,good,http://example.com,96 +94,fine,http://forum.com,57 +53,good,http://news.com,66 +18,fine,https://musicstream.com,54 +81,yes,http://mysite.org,0 +63,fine,https://musicstream.com,57 +9,good,http://example.com,35 +93,bad,http://news.com,84 +28,no,https://videosite.com,4 +66,ok,http://example.com,42 +12,terrible,https://github.com,20 +95,yes,http://news.com,82 +58,excellent,https://yourblog.net,26 +21,terrible,https://musicstream.com,16 +23,excellent,https://github.com,56 +21,bad,https://github.com,33 +90,excellent,http://forum.com,65 +39,yes,https://google.com,76 +18,good,http://mysite.org,81 +44,good,http://example.com,71 +98,fine,http://news.com,21 +20,ok,http://news.com,62 +25,bad,https://shoponline.com,72 +2,fine,https://yourblog.net,37 +47,terrible,https://videosite.com,3 +18,no,https://yourblog.net,54 +32,bad,http://news.com,58 +22,terrible,https://videosite.com,54 +56,yes,https://shoponline.com,17 +56,no,http://forum.com,96 +5,yes,https://github.com,10 +12,bad,http://example.com,41 +22,no,http://example.com,80 +45,good,https://shoponline.com,35 +42,yes,http://mysite.org,24 +36,ok,https://google.com,80 +28,excellent,http://example.com,4 +5,good,https://shoponline.com,81 +68,great,https://yourblog.net,0 +20,yes,http://example.com,30 +75,good,https://yourblog.net,45 +81,no,http://mysite.org,88 +8,no,https://github.com,78 +41,terrible,https://github.com,59 +10,terrible,https://videosite.com,38 +42,fine,http://news.com,91 +27,terrible,https://shoponline.com,2 +9,terrible,http://mysite.org,30 +57,yes,http://forum.com,29 +67,excellent,https://github.com,92 +67,great,https://videosite.com,21 +88,bad,https://google.com,63 +86,terrible,https://github.com,30 +16,fine,http://news.com,80 +77,good,https://videosite.com,95 +16,ok,http://mysite.org,20 +8,terrible,http://example.com,34 +71,excellent,https://google.com,59 +31,maybe,https://shoponline.com,74 +57,no,http://mysite.org,21 +73,fine,https://shoponline.com,49 +88,great,http://forum.com,61 +89,fine,http://mysite.org,38 +84,fine,https://yourblog.net,24 +50,excellent,https://videosite.com,6 +82,good,https://github.com,96 +72,bad,https://yourblog.net,18 +55,no,http://example.com,22 +100,excellent,http://example.com,94 +50,maybe,http://mysite.org,14 +88,fine,http://forum.com,13 +56,ok,https://musicstream.com,69 +46,excellent,https://musicstream.com,14 +22,yes,https://github.com,57 +19,maybe,https://google.com,19 +68,excellent,http://example.com,11 +23,yes,https://videosite.com,64 +19,bad,https://shoponline.com,72 +91,yes,https://github.com,41 +99,good,http://mysite.org,91 +39,bad,https://github.com,46 +10,great,http://example.com,82 +72,bad,http://example.com,92 +63,bad,https://videosite.com,14 +33,great,http://example.com,37 +18,great,https://github.com,3 +92,ok,http://example.com,43 +37,excellent,http://mysite.org,92 +31,maybe,https://musicstream.com,24 +34,fine,https://yourblog.net,73 +27,good,http://mysite.org,32 +20,excellent,https://google.com,42 +56,ok,http://example.com,67 +80,fine,https://videosite.com,71 +94,great,http://forum.com,33 +89,no,https://videosite.com,19 +69,no,https://github.com,46 +10,excellent,https://yourblog.net,55 +31,no,https://musicstream.com,47 +76,bad,http://mysite.org,39 +77,excellent,https://yourblog.net,44 +28,maybe,https://google.com,14 +63,ok,http://news.com,69 +98,bad,https://musicstream.com,3 +52,maybe,https://google.com,70 +42,fine,http://mysite.org,90 +45,no,http://forum.com,69 +69,great,https://github.com,67 +41,maybe,http://mysite.org,100 +25,terrible,http://forum.com,55 +88,no,https://videosite.com,50 +27,good,http://example.com,45 +18,excellent,https://videosite.com,22 +52,ok,http://mysite.org,48 +25,bad,https://google.com,58 +42,maybe,https://musicstream.com,55 +17,no,http://forum.com,87 +57,maybe,https://shoponline.com,23 +5,terrible,https://musicstream.com,65 +15,maybe,http://news.com,63 +74,good,https://shoponline.com,45 +60,terrible,http://news.com,91 +37,fine,http://news.com,97 +57,good,https://github.com,6 +28,no,https://yourblog.net,15 +3,excellent,https://musicstream.com,60 +63,bad,https://yourblog.net,94 +10,terrible,http://example.com,31 +93,no,https://google.com,3 +97,yes,https://shoponline.com,30 +40,ok,http://mysite.org,51 +52,great,https://google.com,80 +25,maybe,https://yourblog.net,85 +48,no,https://musicstream.com,43 +39,bad,http://mysite.org,34 +71,no,https://shoponline.com,90 +30,great,https://yourblog.net,64 +28,maybe,https://shoponline.com,84 +82,excellent,http://example.com,27 +15,bad,http://mysite.org,84 +5,good,http://mysite.org,18 +32,good,http://example.com,58 +97,yes,https://google.com,86 +35,bad,https://shoponline.com,30 +4,excellent,https://videosite.com,31 +10,terrible,https://videosite.com,54 +28,terrible,https://musicstream.com,43 +45,excellent,https://musicstream.com,87 +57,great,http://mysite.org,67 +15,excellent,http://forum.com,5 +91,maybe,http://forum.com,78 +29,good,https://shoponline.com,89 +50,ok,http://mysite.org,4 +98,yes,http://example.com,41 +20,no,https://videosite.com,68 +7,terrible,http://mysite.org,57 +59,yes,http://example.com,26 +87,good,https://github.com,77 +89,fine,http://forum.com,88 +37,no,https://github.com,19 +52,maybe,https://videosite.com,65 +43,maybe,http://example.com,54 +17,excellent,http://forum.com,67 +0,yes,http://example.com,6 +46,terrible,http://news.com,40 +92,great,http://news.com,54 +7,great,https://github.com,66 +94,terrible,https://videosite.com,9 +55,no,https://github.com,8 +99,ok,https://videosite.com,50 +69,fine,https://videosite.com,79 +56,no,https://google.com,87 +53,yes,https://videosite.com,15 +15,ok,http://example.com,59 +87,excellent,http://forum.com,63 +99,fine,https://shoponline.com,33 +59,maybe,https://google.com,90 +38,fine,https://videosite.com,67 +73,terrible,https://shoponline.com,52 +81,terrible,https://github.com,1 +65,excellent,http://mysite.org,74 +60,ok,https://musicstream.com,35 +40,excellent,https://yourblog.net,0 +64,no,http://news.com,83 +74,good,http://news.com,45 +93,no,https://google.com,80 +72,terrible,http://mysite.org,16 +28,excellent,http://mysite.org,26 +81,maybe,http://forum.com,20 +5,yes,https://shoponline.com,68 +61,terrible,https://yourblog.net,52 +90,ok,http://mysite.org,26 +97,no,http://mysite.org,6 +44,terrible,https://google.com,83 +49,no,https://github.com,64 +81,great,http://forum.com,30 +67,good,https://videosite.com,93 +66,ok,https://google.com,66 +20,fine,https://yourblog.net,45 +53,excellent,https://musicstream.com,20 +27,terrible,https://github.com,36 +90,yes,https://videosite.com,67 +72,ok,https://shoponline.com,41 +100,no,https://github.com,39 +59,yes,http://mysite.org,43 +98,good,https://musicstream.com,99 +96,bad,http://forum.com,17 +71,great,https://shoponline.com,50 +80,great,http://example.com,19 +30,excellent,https://github.com,85 +70,bad,http://news.com,61 +6,bad,https://shoponline.com,57 +51,yes,https://shoponline.com,51 +63,no,http://example.com,37 +24,great,http://forum.com,0 +10,yes,http://mysite.org,52 +38,maybe,http://mysite.org,41 +79,maybe,http://example.com,61 +99,fine,http://example.com,25 +76,ok,https://musicstream.com,73 +79,no,http://news.com,97 +32,yes,https://github.com,64 +29,fine,https://shoponline.com,98 +34,excellent,http://forum.com,74 +19,fine,http://example.com,37 +11,great,http://mysite.org,57 +48,great,http://mysite.org,70 +29,yes,http://forum.com,52 +69,no,https://shoponline.com,45 +39,no,https://musicstream.com,59 +42,fine,http://forum.com,50 +10,yes,http://forum.com,95 +1,bad,https://yourblog.net,95 +93,terrible,https://musicstream.com,24 +96,ok,http://example.com,70 +75,ok,https://google.com,45 +60,terrible,https://videosite.com,96 +77,excellent,http://mysite.org,69 +30,fine,https://yourblog.net,77 +13,maybe,https://shoponline.com,90 +25,good,https://github.com,41 +59,no,https://videosite.com,32 +43,excellent,https://github.com,12 +32,maybe,https://google.com,17 +94,maybe,https://shoponline.com,29 +5,great,http://example.com,94 +48,great,http://example.com,59 +98,terrible,https://yourblog.net,71 +9,excellent,https://github.com,85 +52,maybe,http://example.com,44 +25,excellent,https://musicstream.com,1 +88,bad,http://mysite.org,3 +83,bad,https://google.com,78 +86,fine,https://google.com,76 +8,maybe,https://yourblog.net,61 +95,great,https://shoponline.com,39 +86,terrible,https://github.com,31 +48,bad,https://videosite.com,36 +14,fine,https://google.com,61 +27,no,http://news.com,97 +96,ok,http://news.com,88 +38,no,https://musicstream.com,38 +86,no,https://shoponline.com,48 +90,excellent,https://yourblog.net,72 +68,maybe,http://example.com,0 +100,good,http://news.com,68 +43,good,https://yourblog.net,12 +83,great,http://mysite.org,64 +84,bad,https://yourblog.net,20 +94,yes,https://github.com,15 +33,excellent,https://musicstream.com,53 +65,bad,http://forum.com,9 +25,yes,http://example.com,30 +8,maybe,https://videosite.com,83 +66,maybe,http://example.com,86 +27,terrible,https://yourblog.net,40 +7,good,http://news.com,29 +21,good,https://google.com,14 +35,maybe,https://videosite.com,45 +46,great,https://videosite.com,38 +18,yes,https://github.com,40 +90,good,https://videosite.com,84 +82,great,http://example.com,20 +5,excellent,http://forum.com,100 +30,terrible,http://example.com,63 +60,no,https://google.com,93 +72,maybe,https://google.com,99 +53,maybe,http://forum.com,52 +87,great,http://forum.com,71 +51,bad,https://shoponline.com,71 +22,good,http://forum.com,89 +87,great,http://forum.com,90 +58,excellent,https://github.com,93 +62,no,https://videosite.com,2 +86,maybe,https://shoponline.com,81 +51,excellent,http://news.com,1 +20,no,http://forum.com,72 +51,no,http://news.com,35 +57,good,https://yourblog.net,41 +71,great,http://news.com,33 +51,fine,https://videosite.com,2 +15,yes,https://musicstream.com,14 +74,yes,http://news.com,64 +41,ok,http://mysite.org,21 +58,fine,https://musicstream.com,29 +0,terrible,http://news.com,77 +94,fine,https://github.com,91 +66,maybe,http://news.com,47 +97,no,http://news.com,18 +25,maybe,https://github.com,26 +26,bad,https://yourblog.net,21 +25,great,https://google.com,33 +17,fine,https://google.com,83 +20,good,http://example.com,40 +3,fine,https://shoponline.com,16 +29,yes,http://news.com,23 +71,excellent,https://github.com,30 +61,ok,https://yourblog.net,24 +14,bad,http://mysite.org,73 +37,ok,http://news.com,52 +92,great,http://news.com,40 +70,bad,https://yourblog.net,29 +39,terrible,https://videosite.com,9 +23,ok,http://example.com,38 +52,excellent,https://yourblog.net,46 +56,ok,https://musicstream.com,76 +83,yes,http://forum.com,96 +1,no,http://news.com,27 +72,excellent,http://mysite.org,12 +75,no,http://forum.com,97 +21,ok,https://yourblog.net,87 +41,yes,http://forum.com,44 +89,excellent,http://mysite.org,40 +46,good,https://google.com,39 +75,terrible,https://shoponline.com,88 +51,terrible,https://shoponline.com,33 +15,terrible,http://mysite.org,20 +93,great,https://yourblog.net,92 +45,excellent,https://musicstream.com,7 +23,excellent,https://google.com,81 +54,no,http://example.com,90 +45,great,https://yourblog.net,33 +67,yes,https://github.com,52 +20,good,https://github.com,44 +39,excellent,https://musicstream.com,60 +20,maybe,https://google.com,20 +16,yes,http://forum.com,19 +52,bad,https://yourblog.net,35 +40,fine,https://google.com,86 +43,ok,https://videosite.com,17 +69,terrible,https://shoponline.com,8 +86,terrible,https://github.com,45 +17,no,https://google.com,29 +53,fine,https://shoponline.com,4 +31,great,https://shoponline.com,5 +51,maybe,https://videosite.com,92 +21,bad,https://videosite.com,0 +96,bad,http://example.com,82 +89,bad,http://forum.com,97 +73,terrible,http://mysite.org,17 +58,bad,https://videosite.com,18 +13,excellent,https://musicstream.com,47 +9,good,https://google.com,3 +46,ok,https://musicstream.com,75 +47,excellent,https://yourblog.net,15 +49,good,https://musicstream.com,16 +47,terrible,https://yourblog.net,60 +63,good,https://musicstream.com,19 +56,good,https://videosite.com,97 +88,great,http://news.com,75 +46,terrible,http://mysite.org,61 +51,ok,https://google.com,90 +67,excellent,http://example.com,56 +4,yes,https://musicstream.com,67 +60,excellent,http://news.com,24 +29,no,https://musicstream.com,38 +37,ok,https://musicstream.com,51 +31,yes,http://mysite.org,66 +4,bad,https://google.com,90 +41,good,http://news.com,64 +19,excellent,https://musicstream.com,19 +79,maybe,https://musicstream.com,56 +100,no,http://news.com,97 +64,yes,https://yourblog.net,8 +45,fine,http://news.com,87 +10,terrible,https://yourblog.net,52 +50,excellent,https://musicstream.com,82 +45,great,http://mysite.org,96 +10,no,https://musicstream.com,0 +28,excellent,http://news.com,77 +100,good,https://shoponline.com,72 +3,fine,http://news.com,68 +87,good,https://shoponline.com,93 +16,ok,http://news.com,65 +41,maybe,http://forum.com,99 +81,maybe,http://mysite.org,33 +1,maybe,http://mysite.org,45 +33,great,https://google.com,64 +73,bad,https://google.com,56 +67,terrible,http://example.com,14 +0,no,https://yourblog.net,8 +25,good,https://google.com,65 +4,ok,http://example.com,74 +89,maybe,https://videosite.com,86 +32,great,http://forum.com,27 +58,no,http://forum.com,91 +64,good,https://github.com,45 +32,bad,https://github.com,92 +31,maybe,http://news.com,12 +70,good,https://yourblog.net,77 +25,bad,https://google.com,8 +25,bad,https://musicstream.com,82 +49,good,http://mysite.org,95 +11,excellent,http://forum.com,39 +28,no,https://yourblog.net,62 +48,maybe,http://example.com,12 +12,excellent,https://shoponline.com,44 +19,excellent,https://shoponline.com,22 +77,yes,https://yourblog.net,94 +24,fine,http://news.com,20 +95,great,http://forum.com,51 +21,maybe,http://example.com,58 +66,no,https://google.com,94 +32,great,https://videosite.com,92 +26,good,https://github.com,54 +52,terrible,https://videosite.com,54 +24,good,http://example.com,89 +67,fine,https://musicstream.com,24 +64,bad,http://news.com,79 +40,bad,https://yourblog.net,76 +2,excellent,https://videosite.com,51 +59,no,http://mysite.org,39 +64,ok,https://yourblog.net,82 +56,ok,http://example.com,75 +82,maybe,https://google.com,87 +14,good,http://news.com,24 +23,yes,http://forum.com,16 +45,great,https://shoponline.com,51 +91,fine,https://google.com,18 +60,bad,https://musicstream.com,64 +33,fine,https://shoponline.com,66 +83,fine,http://mysite.org,40 +7,yes,https://github.com,84 +9,no,https://musicstream.com,90 +51,ok,https://google.com,32 +83,bad,http://example.com,63 +2,terrible,https://videosite.com,2 +76,terrible,https://yourblog.net,43 +6,yes,https://videosite.com,26 +98,great,https://shoponline.com,34 +38,ok,http://example.com,14 +90,fine,http://mysite.org,44 +21,fine,http://forum.com,17 +20,yes,http://mysite.org,11 +47,no,https://shoponline.com,19 +69,bad,https://videosite.com,60 +30,excellent,http://forum.com,48 +43,great,https://yourblog.net,35 +94,yes,https://google.com,54 +97,good,http://forum.com,63 +62,excellent,https://videosite.com,44 +42,ok,https://shoponline.com,14 +11,no,http://example.com,80 +80,no,https://videosite.com,60 +2,excellent,https://musicstream.com,64 +49,maybe,https://videosite.com,18 +48,maybe,http://mysite.org,88 +46,bad,https://musicstream.com,65 +86,excellent,https://yourblog.net,90 +75,ok,https://google.com,54 +81,ok,http://mysite.org,64 +59,good,https://github.com,74 +24,good,http://mysite.org,84 +97,excellent,http://example.com,38 +73,excellent,http://mysite.org,98 +78,bad,http://example.com,57 +37,fine,https://videosite.com,84 +89,maybe,http://news.com,12 +89,bad,https://yourblog.net,36 +63,terrible,http://example.com,5 +91,no,https://videosite.com,68 +52,fine,http://forum.com,26 +34,no,https://shoponline.com,51 +38,no,https://google.com,66 +78,ok,https://shoponline.com,59 +14,great,https://shoponline.com,90 +93,good,http://mysite.org,31 +96,ok,https://shoponline.com,93 +73,maybe,https://shoponline.com,9 +5,ok,https://shoponline.com,37 +73,excellent,http://example.com,49 +32,no,https://github.com,39 +61,bad,http://example.com,62 +20,ok,https://musicstream.com,55 +33,yes,https://shoponline.com,2 +31,terrible,https://google.com,58 +1,bad,http://forum.com,78 +62,great,https://yourblog.net,38 +80,great,http://mysite.org,54 +6,terrible,http://mysite.org,49 +45,yes,https://videosite.com,13 +97,fine,https://shoponline.com,71 +18,no,http://mysite.org,91 +94,bad,https://yourblog.net,52 +4,excellent,http://example.com,92 +95,great,http://forum.com,100 +99,no,https://yourblog.net,36 +13,terrible,https://github.com,55 +36,bad,https://yourblog.net,82 +18,fine,https://videosite.com,7 +43,good,https://github.com,60 +83,excellent,https://google.com,14 +0,fine,https://shoponline.com,34 +66,bad,https://videosite.com,14 +66,no,http://news.com,70 +29,good,http://news.com,65 +83,yes,http://news.com,49 +71,terrible,http://forum.com,85 +52,yes,http://example.com,55 +59,no,http://mysite.org,94 +30,no,http://forum.com,38 +98,good,http://example.com,89 +52,ok,https://yourblog.net,71 +100,bad,http://example.com,73 +43,maybe,https://musicstream.com,56 +56,no,https://yourblog.net,33 +30,yes,http://forum.com,71 +19,maybe,https://google.com,15 +33,ok,http://news.com,69 +84,excellent,https://shoponline.com,6 +75,fine,https://musicstream.com,70 +95,great,https://musicstream.com,16 +5,terrible,https://github.com,90 +90,yes,https://videosite.com,10 +58,excellent,https://yourblog.net,0 +4,bad,https://musicstream.com,30 +54,good,http://example.com,20 +56,excellent,https://videosite.com,48 +65,fine,http://example.com,16 +61,bad,http://example.com,69 +87,maybe,http://forum.com,0 +61,bad,https://github.com,16 +0,maybe,https://google.com,60 +88,terrible,http://mysite.org,24 +17,maybe,https://github.com,81 +61,excellent,https://musicstream.com,39 +84,excellent,https://yourblog.net,30 +51,yes,http://news.com,99 +84,excellent,http://news.com,79 +56,yes,https://videosite.com,51 +23,yes,https://github.com,75 +30,no,https://github.com,92 +81,excellent,https://videosite.com,67 +93,ok,http://example.com,13 +82,good,http://forum.com,75 +100,maybe,http://example.com,37 +52,terrible,https://musicstream.com,26 +90,yes,https://videosite.com,8 +7,yes,http://news.com,5 +35,excellent,https://github.com,90 +16,terrible,http://forum.com,82 +7,terrible,https://github.com,45 +18,great,http://example.com,18 +29,maybe,https://yourblog.net,3 +46,good,http://forum.com,83 +1,maybe,http://example.com,27 +31,fine,https://videosite.com,29 +10,bad,https://musicstream.com,86 +68,no,http://example.com,19 +17,great,http://forum.com,93 +17,good,https://videosite.com,54 +83,bad,https://musicstream.com,51 +89,yes,https://yourblog.net,58 +42,fine,https://github.com,19 +16,yes,https://google.com,93 +72,good,https://videosite.com,53 +56,yes,https://google.com,39 +70,maybe,https://musicstream.com,82 +3,fine,https://shoponline.com,25 +71,great,https://videosite.com,100 +8,yes,http://mysite.org,91 +20,yes,https://videosite.com,88 +40,fine,http://example.com,31 +65,good,http://forum.com,3 +23,yes,http://forum.com,69 +32,ok,http://mysite.org,94 +38,terrible,http://example.com,23 +80,good,https://github.com,63 +28,yes,https://videosite.com,11 +92,bad,http://forum.com,83 +67,maybe,https://musicstream.com,70 +98,maybe,http://forum.com,4 +34,maybe,https://videosite.com,68 +61,great,https://google.com,9 +83,fine,https://yourblog.net,100 +35,bad,https://yourblog.net,65 +16,fine,https://videosite.com,5 +90,yes,http://mysite.org,8 +78,terrible,https://shoponline.com,95 +4,yes,https://google.com,47 +4,good,https://musicstream.com,13 +56,terrible,http://example.com,96 +97,no,http://example.com,95 +77,maybe,http://news.com,15 +88,maybe,https://google.com,60 +0,fine,https://shoponline.com,14 +72,no,https://musicstream.com,69 +61,terrible,http://example.com,31 +24,yes,https://github.com,85 +68,great,http://example.com,97 +50,great,https://videosite.com,19 +6,excellent,https://yourblog.net,91 +81,excellent,https://yourblog.net,98 +22,no,http://mysite.org,45 +67,yes,https://yourblog.net,53 +60,ok,https://yourblog.net,20 +44,good,https://yourblog.net,37 +73,ok,https://github.com,4 +54,good,http://example.com,14 +69,good,https://yourblog.net,39 +59,excellent,http://example.com,15 +53,terrible,https://google.com,29 +56,no,https://musicstream.com,46 +76,ok,https://shoponline.com,64 +2,good,https://musicstream.com,74 +51,excellent,http://forum.com,66 +17,terrible,http://news.com,8 +66,maybe,https://shoponline.com,92 +26,fine,http://mysite.org,51 +45,excellent,http://news.com,50 +92,yes,https://videosite.com,93 +34,terrible,https://google.com,50 +40,yes,https://shoponline.com,73 +50,bad,http://forum.com,10 +38,bad,https://google.com,28 +1,terrible,http://mysite.org,97 +7,fine,https://shoponline.com,18 +36,excellent,https://videosite.com,73 +44,no,https://yourblog.net,10 +60,ok,https://videosite.com,45 +59,ok,https://videosite.com,81 +93,yes,http://news.com,75 +73,no,https://github.com,44 +39,yes,https://musicstream.com,71 +46,excellent,https://musicstream.com,25 +0,maybe,http://mysite.org,82 +65,bad,http://mysite.org,27 +57,no,http://example.com,60 +29,maybe,https://google.com,48 +68,maybe,http://example.com,34 +33,excellent,http://forum.com,70 +12,maybe,https://yourblog.net,63 +83,maybe,http://forum.com,43 +25,yes,https://yourblog.net,74 +58,terrible,https://yourblog.net,79 +31,no,http://example.com,72 +21,great,http://mysite.org,74 +5,no,http://forum.com,41 +44,fine,https://github.com,75 +63,great,https://github.com,72 +9,good,https://github.com,93 +96,maybe,https://yourblog.net,83 +18,yes,http://example.com,14 +13,terrible,https://github.com,80 +30,bad,https://yourblog.net,65 +69,yes,http://news.com,30 +19,yes,http://mysite.org,96 +91,fine,http://example.com,46 +68,maybe,https://github.com,37 +2,terrible,https://yourblog.net,41 +33,ok,https://shoponline.com,39 +54,terrible,http://news.com,60 +60,maybe,http://forum.com,89 +76,great,http://mysite.org,37 +66,bad,https://google.com,16 +63,terrible,https://musicstream.com,60 +44,no,https://yourblog.net,61 +66,great,http://example.com,16 +4,good,https://github.com,67 +17,great,https://shoponline.com,100 +84,good,http://mysite.org,56 +29,maybe,https://musicstream.com,32 +19,fine,https://shoponline.com,51 +36,good,http://mysite.org,77 +37,terrible,http://mysite.org,95 diff --git a/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_p0.groovy b/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_p0.groovy new file mode 100644 index 00000000000000..d25c4f149eb834 --- /dev/null +++ b/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_p0.groovy @@ -0,0 +1,151 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import java.util.concurrent.Executors +import java.util.concurrent.TimeUnit +import org.awaitility.Awaitility + +suite("test_index_compaction_p0", "p0, nonConcurrent") { + + def compaction_table_name = "httplogs" + + def load_json_data = {table_name, file_name -> + // load the json data + streamLoad { + table "${table_name}" + + // set http request header params + set 'read_json_by_line', 'true' + set 'format', 'json' + set 'max_filter_ratio', '0.1' + file file_name // import json file + time 10000 // limit inflight 10s + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + logger.info("Stream load ${file_name} result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + // assertEquals(json.NumberTotalRows, json.NumberLoadedRows + json.NumberUnselectedRows) + assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) + } + } + } + sql "DROP TABLE IF EXISTS ${compaction_table_name}" + sql """ + CREATE TABLE ${compaction_table_name} ( + `@timestamp` int(11) NULL, + `clientip` varchar(20) NULL, + `request` varchar(500) NULL, + `status` int NULL, + `size` int NULL, + INDEX clientip_idx (`clientip`) USING INVERTED COMMENT '', + INDEX request_idx (`request`) USING INVERTED PROPERTIES("parser" = "unicode") COMMENT '', + INDEX status_idx (`status`) USING INVERTED COMMENT '', + INDEX size_idx (`size`) USING INVERTED COMMENT '' + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`@timestamp`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "compaction_policy" = "time_series", + "time_series_compaction_file_count_threshold" = "10", + "disable_auto_compaction" = "true" + ); + """ + def executor = Executors.newFixedThreadPool(20) + (1..20).each { i -> + executor.submit { + def fileName = "documents-" + i + ".json" + load_json_data.call(compaction_table_name, """${getS3Url()}/regression/inverted_index_cases/httplogs/${fileName}""") + } + } + executor.shutdown() + executor.awaitTermination(1, TimeUnit.MINUTES) + + def backendId_to_backendIP = [:] + def backendId_to_backendHttpPort = [:] + getBackendIpHttpPort(backendId_to_backendIP, backendId_to_backendHttpPort); + + def set_be_config = { key, value -> + for (String backend_id: backendId_to_backendIP.keySet()) { + def (code, out, err) = update_be_config(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), key, value) + logger.info("update config: code=" + code + ", out=" + out + ", err=" + err) + } + } + set_be_config.call("inverted_index_compaction_enable", "true") + //TabletId,ReplicaId,BackendId,SchemaHash,Version,LstSuccessVersion,LstFailedVersion,LstFailedTime,LocalDataSize,RemoteDataSize,RowCount,State,LstConsistencyCheckTime,CheckVersion,VersionCount,QueryHits,PathHash,MetaUrl,CompactionStatus + def tablets = sql_return_maparray """ show tablets from ${compaction_table_name}; """ + + int beforeSegmentCount = 0 + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + beforeSegmentCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + assertEquals(beforeSegmentCount, 20) + + // trigger compactions for all tablets in ${tableName} + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_run_cumulative_compaction(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactJson = parseJson(out.trim()) + assertEquals("success", compactJson.status.toLowerCase()) + } + + // wait for all compactions done + for (def tablet in tablets) { + Awaitility.await().atMost(1, TimeUnit.MINUTES).untilAsserted(() -> { + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("compaction task for this tablet is not running", compactionStatus.msg.toLowerCase()) + return compactionStatus.run_status; + }); + } + + int afterSegmentCount = 0 + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + logger.info("rowset is: " + rowset) + afterSegmentCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + assertEquals(afterSegmentCount, 1) +} diff --git a/regression-test/suites/inverted_index_p1/index_compaction/test_index_compaction_p1.groovy b/regression-test/suites/inverted_index_p1/index_compaction/test_index_compaction_p1.groovy new file mode 100644 index 00000000000000..a64e4da7d82a19 --- /dev/null +++ b/regression-test/suites/inverted_index_p1/index_compaction/test_index_compaction_p1.groovy @@ -0,0 +1,151 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import java.util.concurrent.Executors +import java.util.concurrent.TimeUnit +import org.awaitility.Awaitility + +suite("test_index_compaction_p1", "p1, nonConcurrent") { + + def compaction_table_name = "httplogs" + + def load_json_data = {table_name, file_name -> + // load the json data + streamLoad { + table "${table_name}" + + // set http request header params + set 'read_json_by_line', 'true' + set 'format', 'json' + set 'max_filter_ratio', '0.1' + file file_name // import json file + time 10000 // limit inflight 10s + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + logger.info("Stream load ${file_name} result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + // assertEquals(json.NumberTotalRows, json.NumberLoadedRows + json.NumberUnselectedRows) + assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) + } + } + } + sql "DROP TABLE IF EXISTS ${compaction_table_name}" + sql """ + CREATE TABLE ${compaction_table_name} ( + `@timestamp` int(11) NULL, + `clientip` varchar(20) NULL, + `request` varchar(500) NULL, + `status` int NULL, + `size` int NULL, + INDEX clientip_idx (`clientip`) USING INVERTED COMMENT '', + INDEX request_idx (`request`) USING INVERTED PROPERTIES("parser" = "unicode") COMMENT '', + INDEX status_idx (`status`) USING INVERTED COMMENT '', + INDEX size_idx (`size`) USING INVERTED COMMENT '' + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`@timestamp`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "compaction_policy" = "time_series", + "time_series_compaction_file_count_threshold" = "20", + "disable_auto_compaction" = "true" + ); + """ + def executor = Executors.newFixedThreadPool(50) + (1..110).each { i -> + executor.submit { + def fileName = "documents-" + i + ".json" + load_json_data.call(compaction_table_name, """${getS3Url()}/regression/inverted_index_cases/httplogs/${fileName}""") + } + } + executor.shutdown() + executor.awaitTermination(10, TimeUnit.MINUTES) + + def backendId_to_backendIP = [:] + def backendId_to_backendHttpPort = [:] + getBackendIpHttpPort(backendId_to_backendIP, backendId_to_backendHttpPort); + + def set_be_config = { key, value -> + for (String backend_id: backendId_to_backendIP.keySet()) { + def (code, out, err) = update_be_config(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), key, value) + logger.info("update config: code=" + code + ", out=" + out + ", err=" + err) + } + } + set_be_config.call("inverted_index_compaction_enable", "true") + //TabletId,ReplicaId,BackendId,SchemaHash,Version,LstSuccessVersion,LstFailedVersion,LstFailedTime,LocalDataSize,RemoteDataSize,RowCount,State,LstConsistencyCheckTime,CheckVersion,VersionCount,QueryHits,PathHash,MetaUrl,CompactionStatus + def tablets = sql_return_maparray """ show tablets from ${compaction_table_name}; """ + + int beforeSegmentCount = 0 + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + beforeSegmentCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + assertEquals(beforeSegmentCount, 110) + + // trigger compactions for all tablets in ${tableName} + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_run_cumulative_compaction(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactJson = parseJson(out.trim()) + assertEquals("success", compactJson.status.toLowerCase()) + } + + // wait for all compactions done + for (def tablet in tablets) { + Awaitility.await().atMost(10, TimeUnit.MINUTES).untilAsserted(() -> { + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("compaction task for this tablet is not running", compactionStatus.msg.toLowerCase()) + return compactionStatus.run_status; + }); + } + + int afterSegmentCount = 0 + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + (code, out, err) = curl("GET", tablet.CompactionStatus) + logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def tabletJson = parseJson(out.trim()) + assert tabletJson.rowsets instanceof List + for (String rowset in (List) tabletJson.rowsets) { + logger.info("rowset is: " + rowset) + afterSegmentCount += Integer.parseInt(rowset.split(" ")[1]) + } + } + assertEquals(afterSegmentCount, 1) +} From f6d82a9fb47e07e67a40f0f33ed05a4cde7bb378 Mon Sep 17 00:00:00 2001 From: csun5285 Date: Sun, 10 Nov 2024 15:06:33 +0800 Subject: [PATCH 2/2] fix --- be/src/common/config.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 800eab8fd9ef40..7de013bb7a52ca 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1070,7 +1070,7 @@ DEFINE_Int32(inverted_index_read_buffer_size, "4096"); // tree depth for bkd index DEFINE_Int32(max_depth_in_bkd_tree, "32"); // index compaction -DEFINE_mBool(inverted_index_compaction_enable, "true"); +DEFINE_mBool(inverted_index_compaction_enable, "false"); // Only for debug, do not use in production DEFINE_mBool(debug_inverted_index_compaction, "false"); // index by RAM directory