diff --git a/be/src/olap/iterators.h b/be/src/olap/iterators.h index cbf8f1eca65ae2a..46952376aad8c83 100644 --- a/be/src/olap/iterators.h +++ b/be/src/olap/iterators.h @@ -42,7 +42,7 @@ struct IteratorRowRef; }; namespace segment_v2 { -struct StreamReader; +struct SubstreamIterator; } class StorageReadOptions { diff --git a/be/src/olap/rowset/segment_v2/column_reader.cpp b/be/src/olap/rowset/segment_v2/column_reader.cpp index a3b0cd662e4b549..cf156ca0496a862 100644 --- a/be/src/olap/rowset/segment_v2/column_reader.cpp +++ b/be/src/olap/rowset/segment_v2/column_reader.cpp @@ -1672,8 +1672,6 @@ static void fill_nested_with_defaults(vectorized::MutableColumnPtr& dst, vectorized::MutableColumnPtr& sibling_column, size_t nrows) { const auto* sibling_array = vectorized::check_and_get_column( remove_nullable(sibling_column->get_ptr())); - CHECK(sibling_array) << "Expected array column, but mmet " << sibling_column->get_name(); - ; const auto* dst_array = vectorized::check_and_get_column( remove_nullable(dst->get_ptr())); if (!dst_array || !sibling_array) { diff --git a/be/src/olap/rowset/segment_v2/hierarchical_data_reader.cpp b/be/src/olap/rowset/segment_v2/hierarchical_data_reader.cpp index 364e688829058fe..59a7496505b93dc 100644 --- a/be/src/olap/rowset/segment_v2/hierarchical_data_reader.cpp +++ b/be/src/olap/rowset/segment_v2/hierarchical_data_reader.cpp @@ -55,7 +55,7 @@ Status HierarchicalDataReader::create(std::unique_ptr* reader, if (read_type == ReadType::MERGE_SPARSE) { ColumnIterator* it; RETURN_IF_ERROR(root->data.reader->new_iterator(&it)); - stream_iter->set_root(std::make_unique( + stream_iter->set_root(std::make_unique( root->data.file_column_type->create_column(), std::unique_ptr(it), root->data.file_column_type)); } @@ -97,7 +97,7 @@ Status HierarchicalDataReader::seek_to_ordinal(ordinal_t ord) { Status HierarchicalDataReader::next_batch(size_t* n, vectorized::MutableColumnPtr& dst, bool* has_null) { return process_read( - [&](StreamReader& reader, const vectorized::PathInData& path, + [&](SubstreamIterator& reader, const vectorized::PathInData& path, const vectorized::DataTypePtr& type) { CHECK(reader.inited); RETURN_IF_ERROR(reader.iterator->next_batch(n, reader.column, has_null)); @@ -112,7 +112,7 @@ Status HierarchicalDataReader::next_batch(size_t* n, vectorized::MutableColumnPt Status HierarchicalDataReader::read_by_rowids(const rowid_t* rowids, const size_t count, vectorized::MutableColumnPtr& dst) { return process_read( - [&](StreamReader& reader, const vectorized::PathInData& path, + [&](SubstreamIterator& reader, const vectorized::PathInData& path, const vectorized::DataTypePtr& type) { CHECK(reader.inited); RETURN_IF_ERROR(reader.iterator->read_by_rowids(rowids, count, reader.column)); @@ -134,8 +134,8 @@ Status HierarchicalDataReader::add_stream(const SubcolumnColumnReaders::Node* no RETURN_IF_ERROR(node->data.reader->new_iterator(&it)); std::unique_ptr it_ptr; it_ptr.reset(it); - StreamReader reader(node->data.file_column_type->create_column(), std::move(it_ptr), - node->data.file_column_type); + SubstreamIterator reader(node->data.file_column_type->create_column(), std::move(it_ptr), + node->data.file_column_type); bool added = _substream_reader.add(node->path, std::move(reader)); if (!added) { return Status::InternalError("Failed to add node path {}", node->path.get_path()); diff --git a/be/src/olap/rowset/segment_v2/hierarchical_data_reader.h b/be/src/olap/rowset/segment_v2/hierarchical_data_reader.h index d2a559ed5712607..75ddea4fee53604 100644 --- a/be/src/olap/rowset/segment_v2/hierarchical_data_reader.h +++ b/be/src/olap/rowset/segment_v2/hierarchical_data_reader.h @@ -72,11 +72,11 @@ class HierarchicalDataReader : public ColumnIterator { Status add_stream(const SubcolumnColumnReaders::Node* node); - void set_root(std::unique_ptr&& root) { _root_reader = std::move(root); } + void set_root(std::unique_ptr&& root) { _root_reader = std::move(root); } private: SubstreamReaderTree _substream_reader; - std::unique_ptr _root_reader; + std::unique_ptr _root_reader; size_t _rows_read = 0; vectorized::PathInData _path; @@ -132,17 +132,18 @@ class HierarchicalDataReader : public ColumnIterator { PathsWithColumnAndType non_nested_subcolumns; RETURN_IF_ERROR(tranverse([&](SubstreamReaderTree::Node& node) { MutableColumnPtr column = node.data.column->get_ptr(); - PathInData real_path = node.path.copy_pop_nfront(_path.get_parts().size()); + PathInData relative_path = node.path.copy_pop_nfront(_path.get_parts().size()); if (node.path.has_nested_part()) { CHECK_EQ(getTypeName(remove_nullable(node.data.type)->get_type_id()), getTypeName(TypeIndex::Array)); PathInData parent_path = node.path.get_nested_prefix_path().copy_pop_nfront( _path.get_parts().size()); - nested_subcolumns[parent_path].emplace_back(real_path, column->get_ptr(), + nested_subcolumns[parent_path].emplace_back(relative_path, column->get_ptr(), node.data.type); } else { - non_nested_subcolumns.emplace_back(real_path, column->get_ptr(), node.data.type); + non_nested_subcolumns.emplace_back(relative_path, column->get_ptr(), + node.data.type); } return Status::OK(); })); @@ -156,6 +157,10 @@ class HierarchicalDataReader : public ColumnIterator { entry.type->get_name()); } } + // Iterate nested subcolumns and flatten them, the entry contains the nested subcolumns of the same nested parent + // first we pick the first subcolumn as base array and using it's offset info. Then we flatten all nested subcolumns + // into a new object column and wrap it with array column using the first element offsets.The wrapped array column + // will type the type of ColumnObject::NESTED_TYPE, whih is Nullable>. for (auto& entry : nested_subcolumns) { MutableColumnPtr nested_object = ColumnObject::create(true, false); const auto* base_array = @@ -173,15 +178,15 @@ class HierarchicalDataReader : public ColumnIterator { } const auto* target_array = check_and_get_column(remove_nullable(subcolumn.column).get()); +#ifndef NDEBUG if (!base_array->has_equal_offsets(*target_array)) { return Status::InvalidArgument( "Meet none equal offsets array when flatten nested array, path {}, " "type {}", subcolumn.path.get_path(), subcolumn.type->get_name()); } - MutableColumnPtr flattend_column = check_and_get_column(target_array) - ->get_data_ptr() - ->assume_mutable(); +#endif + MutableColumnPtr flattend_column = target_array->get_data_ptr()->assume_mutable(); DataTypePtr flattend_type = check_and_get_data_type(remove_nullable(type).get()) ->get_nested_type(); @@ -256,7 +261,7 @@ class HierarchicalDataReader : public ColumnIterator { // encodes sparse columns that are not materialized class ExtractReader : public ColumnIterator { public: - ExtractReader(const TabletColumn& col, std::unique_ptr&& root_reader, + ExtractReader(const TabletColumn& col, std::unique_ptr&& root_reader, vectorized::DataTypePtr target_type_hint) : _col(col), _root_reader(std::move(root_reader)), @@ -280,7 +285,7 @@ class ExtractReader : public ColumnIterator { TabletColumn _col; // may shared among different column iterators - std::unique_ptr _root_reader; + std::unique_ptr _root_reader; vectorized::DataTypePtr _target_type_hint; }; diff --git a/be/src/olap/rowset/segment_v2/segment.cpp b/be/src/olap/rowset/segment_v2/segment.cpp index 7b7aa339dc7a6a8..6a97cfab07160ae 100644 --- a/be/src/olap/rowset/segment_v2/segment.cpp +++ b/be/src/olap/rowset/segment_v2/segment.cpp @@ -568,9 +568,9 @@ Status Segment::_new_iterator_with_variant_root(const TabletColumn& tablet_colum RETURN_IF_ERROR(root->data.reader->new_iterator(&it)); auto stream_iter = new ExtractReader( tablet_column, - std::make_unique(root->data.file_column_type->create_column(), - std::unique_ptr(it), - root->data.file_column_type), + std::make_unique(root->data.file_column_type->create_column(), + std::unique_ptr(it), + root->data.file_column_type), target_type_hint); iter->reset(stream_iter); return Status::OK(); @@ -604,42 +604,40 @@ Status Segment::new_column_iterator_with_path(const TabletColumn& tablet_column, type == ReaderType::READER_FULL_COMPACTION || type == ReaderType::READER_CHECKSUM; }; - auto new_default_iter = [&]() { - if (tablet_column.is_nested_subcolumn() && - type_to_read_flat_leaves(opt->io_ctx.reader_type)) { - // We find node that represents the same Nested type as path. - const auto* parent = _sub_column_tree.find_best_match(*tablet_column.path_info_ptr()); - VLOG_DEBUG << "find with path " << tablet_column.path_info_ptr()->get_path() - << " parent " << (parent ? parent->path.get_path() : "nullptr") << ", type " - << ", parent is nested " << (parent ? parent->is_nested() : false) << ", " - << TabletColumn::get_string_by_field_type(tablet_column.type()); - // find it's common parent with nested part - // why not use parent->path->has_nested_part? because parent may not be a leaf node - // none leaf node may not contain path info - // Example: - // {"payload" : {"commits" : [{"issue" : {"id" : 123, "email" : "a@b"}}]}} - // nested node path : payload.commits(NESTED) - // tablet_column path_info : payload.commits.issue.id(SCALAR - // parent path node : payload.commits.issue(TUPLE) - // leaf path_info : payload.commits.issue.email(SCALAR) - if (parent && SubcolumnColumnReaders::find_parent( - parent, [](const auto& node) { return node.is_nested(); })) { - /// Find any leaf of Nested subcolumn. - const auto* leaf = SubcolumnColumnReaders::find_leaf( - parent, [](const auto& node) { return node.path.has_nested_part(); }); - assert(leaf); - std::unique_ptr sibling_iter; - ColumnIterator* sibling_iter_ptr; - RETURN_IF_ERROR(leaf->data.reader->new_iterator(&sibling_iter_ptr)); - sibling_iter.reset(sibling_iter_ptr); - *iter = std::make_unique(std::move(sibling_iter), - leaf->data.file_column_type); - } else { - *iter = std::make_unique(nullptr, nullptr); - } - return Status::OK(); + // find the sibling of the nested column to fill the target nested column + auto new_default_iter_with_same_nested = [&](const TabletColumn& tablet_column, + std::unique_ptr* iter) { + // We find node that represents the same Nested type as path. + const auto* parent = _sub_column_tree.find_best_match(*tablet_column.path_info_ptr()); + VLOG_DEBUG << "find with path " << tablet_column.path_info_ptr()->get_path() << " parent " + << (parent ? parent->path.get_path() : "nullptr") << ", type " + << ", parent is nested " << (parent ? parent->is_nested() : false) << ", " + << TabletColumn::get_string_by_field_type(tablet_column.type()); + // find it's common parent with nested part + // why not use parent->path->has_nested_part? because parent may not be a leaf node + // none leaf node may not contain path info + // Example: + // {"payload" : {"commits" : [{"issue" : {"id" : 123, "email" : "a@b"}}]}} + // nested node path : payload.commits(NESTED) + // tablet_column path_info : payload.commits.issue.id(SCALAR) + // parent path node : payload.commits.issue(TUPLE) + // leaf path_info : payload.commits.issue.email(SCALAR) + if (parent && SubcolumnColumnReaders::find_parent( + parent, [](const auto& node) { return node.is_nested(); })) { + /// Find any leaf of Nested subcolumn. + const auto* leaf = SubcolumnColumnReaders::find_leaf( + parent, [](const auto& node) { return node.path.has_nested_part(); }); + assert(leaf); + std::unique_ptr sibling_iter; + ColumnIterator* sibling_iter_ptr; + RETURN_IF_ERROR(leaf->data.reader->new_iterator(&sibling_iter_ptr)); + sibling_iter.reset(sibling_iter_ptr); + *iter = std::make_unique(std::move(sibling_iter), + leaf->data.file_column_type); + } else { + *iter = std::make_unique(nullptr, nullptr); } - return new_default_iterator(tablet_column, iter); + return Status::OK(); }; if (opt != nullptr && type_to_read_flat_leaves(opt->io_ctx.reader_type)) { @@ -653,7 +651,12 @@ Status Segment::new_column_iterator_with_path(const TabletColumn& tablet_column, RETURN_IF_ERROR(_new_iterator_with_variant_root( tablet_column, iter, root, sparse_node->data.file_column_type)); } else { - RETURN_IF_ERROR(new_default_iter()); + if (tablet_column.is_nested_subcolumn()) { + // using the sibling of the nested column to fill the target nested column + RETURN_IF_ERROR(new_default_iter_with_same_nested(tablet_column, iter)); + } else { + RETURN_IF_ERROR(new_default_iterator(tablet_column, iter)); + } } return Status::OK(); } @@ -690,7 +693,7 @@ Status Segment::new_column_iterator_with_path(const TabletColumn& tablet_column, sparse_node->data.file_column_type)); } else { // No such variant column in this segment, get a default one - RETURN_IF_ERROR(new_default_iter()); + RETURN_IF_ERROR(new_default_iterator(tablet_column, iter)); } } diff --git a/be/src/olap/rowset/segment_v2/stream_reader.h b/be/src/olap/rowset/segment_v2/stream_reader.h index f7337268d1633c2..9aac3c0f2328956 100644 --- a/be/src/olap/rowset/segment_v2/stream_reader.h +++ b/be/src/olap/rowset/segment_v2/stream_reader.h @@ -23,21 +23,27 @@ namespace doris::segment_v2 { -struct StreamReader { +// This file Defined ColumnIterator and ColumnReader for reading variant subcolumns. The types from read schema and from storage are +// different, so we need to wrap the ColumnIterator from execution phase and storage column reading phase.And we also +// maintain the tree structure to get the full JSON structure for variants. + +// Wrapped ColumnIterator from execution phase, the type is from read schema +struct SubstreamIterator { vectorized::MutableColumnPtr column; std::unique_ptr iterator; std::shared_ptr type; bool inited = false; size_t rows_read = 0; - StreamReader() = default; - StreamReader(vectorized::MutableColumnPtr&& col, std::unique_ptr&& it, - std::shared_ptr t) + SubstreamIterator() = default; + SubstreamIterator(vectorized::MutableColumnPtr&& col, std::unique_ptr&& it, + std::shared_ptr t) : column(std::move(col)), iterator(std::move(it)), type(t) {} }; // path -> StreamReader -using SubstreamReaderTree = vectorized::SubcolumnsTree; +using SubstreamReaderTree = vectorized::SubcolumnsTree; +// Reader for the storage layer, the file_column_type indicates the read type of the column in segment file struct SubcolumnReader { std::unique_ptr reader; std::shared_ptr file_column_type; diff --git a/be/src/vec/columns/column_dummy.h b/be/src/vec/columns/column_dummy.h index 07bcf55bad88686..b8c2363fe6a0a33 100644 --- a/be/src/vec/columns/column_dummy.h +++ b/be/src/vec/columns/column_dummy.h @@ -64,7 +64,7 @@ class IColumnDummy : public IColumn { void insert_data(const char*, size_t) override { ++s; } - void clear() override {}; + void clear() override { s = 0; } StringRef serialize_value_into_arena(size_t /*n*/, Arena& arena, char const*& begin) const override { diff --git a/be/src/vec/columns/column_object.cpp b/be/src/vec/columns/column_object.cpp index c38606155ed705f..7a77d8f845e8947 100644 --- a/be/src/vec/columns/column_object.cpp +++ b/be/src/vec/columns/column_object.cpp @@ -484,7 +484,7 @@ Array create_empty_array_field(size_t num_dimensions) { static ColumnPtr recreate_column_with_default_values(const ColumnPtr& column, TypeIndex scalar_type, size_t num_dimensions) { const auto* column_array = check_and_get_column(remove_nullable(column).get()); - if (column_array && num_dimensions) { + if (column_array != nullptr && num_dimensions != 0) { return make_nullable(ColumnArray::create( recreate_column_with_default_values(column_array->get_data_ptr(), scalar_type, num_dimensions - 1), @@ -496,7 +496,7 @@ static ColumnPtr recreate_column_with_default_values(const ColumnPtr& column, Ty ->clone_resized(column->size()); } -ColumnObject::Subcolumn ColumnObject::Subcolumn::recreate_with_default_values( +ColumnObject::Subcolumn ColumnObject::Subcolumn::clone_with_default_values( const FieldInfo& field_info) const { Subcolumn new_subcolumn(*this); new_subcolumn.least_common_type = @@ -998,7 +998,7 @@ void ColumnObject::get(size_t n, Field& res) const { for (const auto& entry : subcolumns) { Field field; entry->data.get(n, field); - // Notice: we treat null as empty field + // Notice: we treat null as empty field, since we do not distinguish null and empty for Variant type. if (field.get_type() != Field::Types::Null) { object.try_emplace(entry->path.get_path(), field); } @@ -1019,13 +1019,15 @@ void ColumnObject::add_nested_subcolumn(const PathInData& key, const FieldInfo& /// We find node that represents the same Nested type as @key. const auto* nested_node = subcolumns.find_best_match(key); - if (nested_node && nested_node->is_nested()) { + // If we have a nested subcolumn and it contains nested node in it's path + if (nested_node && + Subcolumns::find_parent(nested_node, [](const auto& node) { return node.is_nested(); })) { /// Find any leaf of Nested subcolumn. const auto* leaf = Subcolumns::find_leaf(nested_node, [&](const auto&) { return true; }); assert(leaf); /// Recreate subcolumn with default values and the same sizes of arrays. - auto new_subcolumn = leaf->data.recreate_with_default_values(field_info); + auto new_subcolumn = leaf->data.clone_with_default_values(field_info); /// It's possible that we have already inserted value from current row /// to this subcolumn. So, adjust size to expected. @@ -1970,7 +1972,7 @@ bool ColumnObject::try_insert_many_defaults_from_nested(const Subcolumns::NodePt /// and replace scalar values to the correct /// default values for given entry. auto new_subcolumn = leaf->data.cut(old_size, leaf->data.size() - old_size) - .recreate_with_default_values(field_info); + .clone_with_default_values(field_info); entry->data.insert_range_from(new_subcolumn, 0, new_subcolumn.size()); return true; diff --git a/be/src/vec/columns/column_object.h b/be/src/vec/columns/column_object.h index 733e2fc5b3a7145..53f4ea08cc11f09 100644 --- a/be/src/vec/columns/column_object.h +++ b/be/src/vec/columns/column_object.h @@ -97,7 +97,8 @@ class ColumnObject final : public COWHelper { constexpr static TypeIndex MOST_COMMON_TYPE_ID = TypeIndex::JSONB; // Nullable(Array(Nullable(Object))) const static DataTypePtr NESTED_TYPE; - // Finlize mode for subcolumns, write mode will deal with sparse columns, only affects in flush block to segments. + // Finlize mode for subcolumns, write mode will estimate which subcolumns are sparse columns(too many null values inside column), + // merge and encode them into a shared column in root column. Only affects in flush block to segments. // Otherwise read mode should be as default mode. enum class FinalizeMode { WRITE_MODE, READ_MODE }; class Subcolumn { @@ -146,10 +147,11 @@ class ColumnObject final : public COWHelper { /// Recreates subcolumn with default scalar values and keeps sizes of arrays. /// Used to create columns of type Nested with consistent array sizes. - Subcolumn recreate_with_default_values(const FieldInfo& field_info) const; + Subcolumn clone_with_default_values(const FieldInfo& field_info) const; void pop_back(size_t n); + // Cut a new subcolumns from current one, element from start to start + length Subcolumn cut(size_t start, size_t length) const; /// Converts all column's parts to the common type and @@ -325,7 +327,6 @@ class ColumnObject final : public COWHelper { /// Adds a subcolumn of specific size with default values. bool add_sub_column(const PathInData& key, size_t new_size); - /// Adds a subcolumn of type Nested of specific size with default values. /// It cares about consistency of sizes of Nested arrays. void add_nested_subcolumn(const PathInData& key, const FieldInfo& field_info, size_t new_size); diff --git a/be/src/vec/data_types/convert_field_to_type.cpp b/be/src/vec/data_types/convert_field_to_type.cpp index 8733929d1b57310..ecbce03ba6b10a9 100644 --- a/be/src/vec/data_types/convert_field_to_type.cpp +++ b/be/src/vec/data_types/convert_field_to_type.cpp @@ -53,7 +53,6 @@ namespace doris::vectorized { * If not, return Field(Null). */ -/** simple types of implementation of visitor to string*/ // TODO support more types class FieldVisitorToStringSimple : public StaticVisitor { public: @@ -63,44 +62,34 @@ class FieldVisitorToStringSimple : public StaticVisitor { String operator()(const Float64& x) const { return std::to_string(x); } String operator()(const String& x) const { return x; } [[noreturn]] String operator()(const UInt128& x) const { - LOG(FATAL) << "not implemeted"; - __builtin_unreachable(); + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "Not implemeted"); } [[noreturn]] String operator()(const Array& x) const { - LOG(FATAL) << "not implemeted"; - __builtin_unreachable(); + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "Not implemeted"); } [[noreturn]] String operator()(const Tuple& x) const { - LOG(FATAL) << "not implemeted"; - __builtin_unreachable(); + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "Not implemeted"); } [[noreturn]] String operator()(const DecimalField& x) const { - LOG(FATAL) << "not implemeted"; - __builtin_unreachable(); + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "Not implemeted"); } [[noreturn]] String operator()(const DecimalField& x) const { - LOG(FATAL) << "not implemeted"; - __builtin_unreachable(); + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "Not implemeted"); } [[noreturn]] String operator()(const DecimalField& x) const { - LOG(FATAL) << "not implemeted"; - __builtin_unreachable(); + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "Not implemeted"); } [[noreturn]] String operator()(const DecimalField& x) const { - LOG(FATAL) << "not implemeted"; - __builtin_unreachable(); + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "Not implemeted"); } [[noreturn]] String operator()(const DecimalField& x) const { - LOG(FATAL) << "not implemeted"; - __builtin_unreachable(); + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "Not implemeted"); } [[noreturn]] String operator()(const JsonbField& x) const { - LOG(FATAL) << "not implemeted"; - __builtin_unreachable(); + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "Not implemeted"); } [[noreturn]] String operator()(const VariantMap& x) const { - LOG(FATAL) << "not implemeted"; - __builtin_unreachable(); + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "Not implemeted"); } }; diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h index 30aacf578c439a7..92ff1961dd71d32 100644 --- a/be/src/vec/functions/function_cast.h +++ b/be/src/vec/functions/function_cast.h @@ -815,11 +815,12 @@ struct ConvertNothingToJsonb { const size_t result, size_t input_rows_count) { const auto& col_with_type_and_name = block.get_by_position(arguments[0]); const IColumn& col_from = *col_with_type_and_name.column; + auto data_type_to = block.get_by_position(result).type; size_t size = col_from.size(); - auto col_to = col_from.clone_resized(size); + auto col_to = data_type_to->create_column_const_with_default_value(size); ColumnUInt8::MutablePtr col_null_map_to = ColumnUInt8::create(size, 1); - block.replace_by_position( - result, ColumnNullable::create(std::move(col_to), std::move(col_null_map_to))); + block.replace_by_position(result, ColumnNullable::create(col_to->assume_mutable(), + std::move(col_null_map_to))); return Status::OK(); } };