diff --git a/be/src/common/config.h b/be/src/common/config.h index 0eb8a600a948fb..7dd3345a7016f5 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -903,6 +903,8 @@ CONF_String(inverted_index_dict_path, "${DORIS_HOME}/dict"); CONF_Int32(max_depth_in_bkd_tree, "32"); // use num_broadcast_buffer blocks as buffer to do broadcast CONF_Int32(num_broadcast_buffer, "32"); +// semi-structure configs +CONF_Bool(enable_parse_multi_dimession_array, "true"); #ifdef BE_TEST // test s3 CONF_String(test_s3_resource, "resource"); diff --git a/be/src/common/consts.h b/be/src/common/consts.h index bf7a2e6013cd66..0f8ce9bdd8469e 100644 --- a/be/src/common/consts.h +++ b/be/src/common/consts.h @@ -27,6 +27,7 @@ const std::string CSV_WITH_NAMES_AND_TYPES = "csv_with_names_and_types"; const std::string BLOCK_TEMP_COLUMN_PREFIX = "__TEMP__"; const std::string ROWID_COL = "__DORIS_ROWID_COL__"; const std::string ROW_STORE_COL = "__DORIS_ROW_STORE_COL__"; +const std::string DYNAMIC_COLUMN_NAME = "__DORIS_DYNAMIC_COL__"; constexpr int MAX_DECIMAL32_PRECISION = 9; constexpr int MAX_DECIMAL64_PRECISION = 18; diff --git a/be/src/common/status.h b/be/src/common/status.h index 91cabd58857edb..33e1e8ea645fbd 100644 --- a/be/src/common/status.h +++ b/be/src/common/status.h @@ -409,6 +409,8 @@ class Status { ErrorCode::TEST_FILE_ERROR == _code || ErrorCode::ROWBLOCK_READ_INFO_ERROR == _code; } + bool is_invalid_argument() const { return ErrorCode::INVALID_ARGUMENT == _code; } + bool is_not_found() const { return _code == ErrorCode::NOT_FOUND; } // Convert into TStatus. Call this if 'status_container' contains an optional diff --git a/be/src/exec/base_scanner.cpp b/be/src/exec/base_scanner.cpp index eb3075f481e9d3..edb057a55d31db 100644 --- a/be/src/exec/base_scanner.cpp +++ b/be/src/exec/base_scanner.cpp @@ -19,6 +19,7 @@ #include +#include "common/consts.h" #include "common/utils.h" #include "exec/exec_node.h" #include "runtime/descriptors.h" @@ -51,6 +52,7 @@ BaseScanner::BaseScanner(RuntimeState* state, RuntimeProfile* profile, _scanner_eof(false) {} Status BaseScanner::open() { + _full_base_schema_view.reset(new vectorized::schema_util::FullBaseSchemaView); RETURN_IF_ERROR(init_expr_ctxes()); if (_params.__isset.strict_mode) { _strict_mode = _params.strict_mode; @@ -102,6 +104,11 @@ Status BaseScanner::init_expr_ctxes() { return Status::InternalError("Unknown source slot descriptor, slot_id={}", slot_id); } _src_slot_descs.emplace_back(it->second); + + if (it->second->type().is_variant_type() && + it->second->col_name() == BeConsts::DYNAMIC_COLUMN_NAME) { + _is_dynamic_schema = true; + } } _row_desc.reset(new RowDescriptor(_state->desc_tbl(), std::vector({_params.src_tuple_id}), @@ -156,6 +163,11 @@ Status BaseScanner::init_expr_ctxes() { } } } + if (_dest_tuple_desc->table_desc()) { + _full_base_schema_view->db_name = _dest_tuple_desc->table_desc()->database(); + _full_base_schema_view->table_name = _dest_tuple_desc->table_desc()->name(); + _full_base_schema_view->table_id = _dest_tuple_desc->table_desc()->table_id(); + } return Status::OK(); } @@ -181,6 +193,9 @@ Status BaseScanner::_materialize_dest_block(vectorized::Block* dest_block) { if (!slot_desc->is_materialized()) { continue; } + if (slot_desc->type().is_variant_type()) { + continue; + } int dest_index = ctx_idx++; auto* ctx = _dest_vexpr_ctx[dest_index]; @@ -250,6 +265,28 @@ Status BaseScanner::_materialize_dest_block(vectorized::Block* dest_block) { std::move(column_ptr), slot_desc->get_data_type_ptr(), slot_desc->col_name())); } + // handle dynamic generated columns + if (!_full_base_schema_view->empty()) { + assert(_is_dynamic_schema); + for (size_t x = dest_block->columns(); x < _src_block.columns(); ++x) { + auto& column_type_name = _src_block.get_by_position(x); + const TColumn& tcolumn = + _full_base_schema_view->column_name_to_column[column_type_name.name]; + auto original_type = vectorized::DataTypeFactory::instance().create_data_type(tcolumn); + // type conflict free path, always cast to original type + if (!column_type_name.type->equals(*original_type)) { + vectorized::ColumnPtr column_ptr; + RETURN_IF_ERROR(vectorized::schema_util::cast_column(column_type_name, + original_type, &column_ptr)); + column_type_name.column = column_ptr; + column_type_name.type = original_type; + } + dest_block->insert(vectorized::ColumnWithTypeAndName(std::move(column_type_name.column), + std::move(column_type_name.type), + column_type_name.name)); + } + } + // after do the dest block insert operation, clear _src_block to remove the reference of origin column if (_src_block_mem_reuse) { _src_block.clear_column_data(origin_column_num); diff --git a/be/src/exec/base_scanner.h b/be/src/exec/base_scanner.h index 972682a0c13a25..8fbcb8b01fdc14 100644 --- a/be/src/exec/base_scanner.h +++ b/be/src/exec/base_scanner.h @@ -19,6 +19,7 @@ #include "common/status.h" #include "util/runtime_profile.h" +#include "vec/common/schema_util.h" #include "vec/exprs/vexpr.h" #include "vec/exprs/vexpr_context.h" @@ -65,6 +66,8 @@ class BaseScanner { // Close this scanner virtual void close() = 0; + bool is_dynamic_schema() const { return _is_dynamic_schema; } + protected: Status _fill_dest_block(vectorized::Block* dest_block, bool* eof); virtual Status _init_src_block(); @@ -124,6 +127,10 @@ class BaseScanner { // slot_ids for parquet predicate push down are in tuple desc TupleId _tupleId = -1; + bool _is_dynamic_schema = false; + // for tracing dynamic schema + std::unique_ptr _full_base_schema_view; + private: Status _filter_src_block(); void _fill_columns_from_path(); diff --git a/be/src/exec/tablet_info.cpp b/be/src/exec/tablet_info.cpp index 0bd8ec87748f64..b833831b235014 100644 --- a/be/src/exec/tablet_info.cpp +++ b/be/src/exec/tablet_info.cpp @@ -84,6 +84,7 @@ Status OlapTableSchemaParam::init(const TOlapTableSchemaParam& tschema) { _db_id = tschema.db_id; _table_id = tschema.table_id; _version = tschema.version; + _is_dynamic_schema = tschema.is_dynamic_schema; std::map slots_map; _tuple_desc = _obj_pool.add(new TupleDescriptor(tschema.tuple_desc)); for (auto& t_slot_desc : tschema.slot_descs) { diff --git a/be/src/exec/tablet_info.h b/be/src/exec/tablet_info.h index e21081544916e0..c1138245e1aa64 100644 --- a/be/src/exec/tablet_info.h +++ b/be/src/exec/tablet_info.h @@ -70,6 +70,8 @@ class OlapTableSchemaParam { return _proto_schema; } + bool is_dynamic_schema() const { return _is_dynamic_schema; } + std::string debug_string() const; private: @@ -81,6 +83,7 @@ class OlapTableSchemaParam { mutable POlapTableSchemaParam* _proto_schema = nullptr; std::vector _indexes; mutable ObjectPool _obj_pool; + bool _is_dynamic_schema = false; }; using OlapTableIndexTablets = TOlapTableIndexTablets; @@ -90,6 +93,7 @@ using OlapTableIndexTablets = TOlapTableIndexTablets; // } using BlockRow = std::pair; +using VecBlock = vectorized::Block; struct VOlapTablePartition { int64_t id = 0; diff --git a/be/src/olap/delta_writer.cpp b/be/src/olap/delta_writer.cpp index 678d5006a0dda9..91d1efa17f7c67 100644 --- a/be/src/olap/delta_writer.cpp +++ b/be/src/olap/delta_writer.cpp @@ -140,6 +140,7 @@ Status DeltaWriter::init() { context.newest_write_timestamp = UnixSeconds(); context.tablet_id = _tablet->table_id(); context.is_direct_write = true; + context.tablet = _tablet; RETURN_NOT_OK(_tablet->create_rowset_writer(context, &_rowset_writer)); _schema.reset(new Schema(_tablet_schema)); _reset_mem_table(); @@ -457,6 +458,8 @@ void DeltaWriter::_build_current_tablet_schema(int64_t index_id, if (_tablet_schema->schema_version() > ori_tablet_schema.schema_version()) { _tablet->update_max_version_schema(_tablet_schema); } + + _tablet_schema->set_table_id(table_schema_param->table_id()); } void DeltaWriter::_request_slave_tablet_pull_rowset(PNodeInfo node_info) { diff --git a/be/src/olap/memtable.cpp b/be/src/olap/memtable.cpp index 9ec3cb0fda2969..8c323fad95ca2f 100644 --- a/be/src/olap/memtable.cpp +++ b/be/src/olap/memtable.cpp @@ -149,7 +149,13 @@ int MemTable::RowInBlockComparator::operator()(const RowInBlock* left, void MemTable::insert(const vectorized::Block* input_block, const std::vector& row_idxs) { SCOPED_CONSUME_MEM_TRACKER(_insert_mem_tracker_use_hook.get()); - auto target_block = input_block->copy_block(_column_offset); + vectorized::Block target_block = *input_block; + if (!_tablet_schema->is_dynamic_schema()) { + // This insert may belong to a rollup tablet, rollup columns is a subset of base table + // but for dynamic table, it's need full columns, so input_block should ignore _column_offset + // of each column and avoid copy_block + target_block = input_block->copy_block(_column_offset); + } if (_is_first_insertion) { _is_first_insertion = false; auto cloneBlock = target_block.clone_without_columns(); @@ -159,6 +165,13 @@ void MemTable::insert(const vectorized::Block* input_block, const std::vectoris_dynamic_schema()) { + // Set _input_mutable_block to dynamic since + // input blocks may be structure-variable(dyanmic) + // this will align _input_mutable_block with + // input_block and auto extends columns + _input_mutable_block.set_block_type(vectorized::BlockType::DYNAMIC); + } } auto num_rows = row_idxs.size(); diff --git a/be/src/olap/olap_common.h b/be/src/olap/olap_common.h index dfc01b85087d13..07b0b289830781 100644 --- a/be/src/olap/olap_common.h +++ b/be/src/olap/olap_common.h @@ -141,6 +141,7 @@ enum FieldType { OLAP_FIELD_TYPE_DECIMAL64 = 32, OLAP_FIELD_TYPE_DECIMAL128I = 33, OLAP_FIELD_TYPE_JSONB = 34, + OLAP_FIELD_TYPE_VARIANT = 35 }; // Define all aggregation methods supported by Field diff --git a/be/src/olap/rowset/beta_rowset_writer.cpp b/be/src/olap/rowset/beta_rowset_writer.cpp index 6d96b8445bdb65..7c19b16a18ab0c 100644 --- a/be/src/olap/rowset/beta_rowset_writer.cpp +++ b/be/src/olap/rowset/beta_rowset_writer.cpp @@ -35,6 +35,8 @@ #include "olap/storage_engine.h" #include "runtime/exec_env.h" #include "runtime/memory/mem_tracker_limiter.h" +#include "vec/common/schema_util.h" // LocalSchemaChangeRecorder +#include "vec/jsonb/serialize.h" namespace doris { using namespace ErrorCode; @@ -98,6 +100,8 @@ Status BetaRowsetWriter::init(const RowsetWriterContext& rowset_writer_context) } _rowset_meta->set_tablet_uid(_context.tablet_uid); _rowset_meta->set_tablet_schema(_context.tablet_schema); + _context.schema_change_recorder = + std::make_shared(); return Status::OK(); } @@ -107,7 +111,7 @@ Status BetaRowsetWriter::add_block(const vectorized::Block* block) { return Status::OK(); } if (UNLIKELY(_segment_writer == nullptr)) { - RETURN_NOT_OK(_create_segment_writer(&_segment_writer)); + RETURN_NOT_OK(_create_segment_writer(&_segment_writer, block)); } return _add_block(block, &_segment_writer); } @@ -489,7 +493,7 @@ bool BetaRowsetWriter::_check_and_set_is_doing_segcompaction() { Status BetaRowsetWriter::_segcompaction_if_necessary() { Status status = Status::OK(); if (!config::enable_segcompaction || !config::enable_storage_vectorization || - !_check_and_set_is_doing_segcompaction()) { + _context.tablet_schema->is_dynamic_schema() || !_check_and_set_is_doing_segcompaction()) { return status; } if (_segcompaction_status.load() != OK) { @@ -557,7 +561,7 @@ Status BetaRowsetWriter::_add_block(const vectorized::Block* block, if (UNLIKELY(max_row_add < 1)) { // no space for another signle row, need flush now RETURN_NOT_OK(_flush_segment_writer(segment_writer)); - RETURN_NOT_OK(_create_segment_writer(segment_writer)); + RETURN_NOT_OK(_create_segment_writer(segment_writer, block)); max_row_add = (*segment_writer)->max_row_to_add(row_avg_size_in_bytes); DCHECK(max_row_add > 0); } @@ -621,7 +625,7 @@ Status BetaRowsetWriter::flush_single_memtable(const vectorized::Block* block, i } RETURN_NOT_OK(_segcompaction_if_necessary()); std::unique_ptr writer; - RETURN_NOT_OK(_create_segment_writer(&writer)); + RETURN_NOT_OK(_create_segment_writer(&writer, block)); RETURN_NOT_OK(_add_block(block, &writer)); RETURN_NOT_OK(_flush_segment_writer(&writer, flush_size)); return Status::OK(); @@ -700,6 +704,23 @@ RowsetSharedPtr BetaRowsetWriter::build() { _rowset_meta->set_newest_write_timestamp(UnixSeconds()); } + // schema changed during this load + if (_context.schema_change_recorder->has_extended_columns()) { + DCHECK(_context.tablet_schema->is_dynamic_schema()) + << "Load can change local schema only in dynamic table"; + TabletSchemaSPtr new_schema = std::make_shared(); + new_schema->copy_from(*_context.tablet_schema); + for (auto const& [_, col] : _context.schema_change_recorder->copy_extended_columns()) { + new_schema->append_column(col); + } + new_schema->set_schema_version(_context.schema_change_recorder->schema_version()); + if (_context.schema_change_recorder->schema_version() > + _context.tablet_schema->schema_version()) { + _context.tablet->update_max_version_schema(new_schema); + } + _rowset_meta->set_tablet_schema(new_schema); + } + RowsetSharedPtr rowset; status = RowsetFactory::create_rowset(_context.tablet_schema, _context.rowset_dir, _rowset_meta, &rowset); @@ -801,7 +822,7 @@ RowsetSharedPtr BetaRowsetWriter::build_tmp() { Status BetaRowsetWriter::_do_create_segment_writer( std::unique_ptr* writer, bool is_segcompaction, int64_t begin, - int64_t end) { + int64_t end, const vectorized::Block* block) { std::string path; int32_t segment_id = 0; if (is_segcompaction) { @@ -847,7 +868,7 @@ Status BetaRowsetWriter::_do_create_segment_writer( } } - auto s = (*writer)->init(); + auto s = (*writer)->init(block); if (!s.ok()) { LOG(WARNING) << "failed to init segment writer: " << s.to_string(); writer->reset(nullptr); @@ -856,8 +877,8 @@ Status BetaRowsetWriter::_do_create_segment_writer( return Status::OK(); } -Status BetaRowsetWriter::_create_segment_writer( - std::unique_ptr* writer) { +Status BetaRowsetWriter::_create_segment_writer(std::unique_ptr* writer, + const vectorized::Block* block) { size_t total_segment_num = _num_segment - _segcompacted_point + 1 + _num_segcompacted; if (UNLIKELY(total_segment_num > config::max_segment_num_per_rowset)) { LOG(WARNING) << "too many segments in rowset." @@ -868,7 +889,7 @@ Status BetaRowsetWriter::_create_segment_writer( << " _num_segcompacted:" << _num_segcompacted; return Status::Error(); } else { - return _do_create_segment_writer(writer, false, -1, -1); + return _do_create_segment_writer(writer, false, -1, -1, block); } } diff --git a/be/src/olap/rowset/beta_rowset_writer.h b/be/src/olap/rowset/beta_rowset_writer.h index ccef9212ffb60d..7f800b31995fbe 100644 --- a/be/src/olap/rowset/beta_rowset_writer.h +++ b/be/src/olap/rowset/beta_rowset_writer.h @@ -32,6 +32,9 @@ class FileWriter; using SegCompactionCandidates = std::vector; using SegCompactionCandidatesSharedPtr = std::shared_ptr; +namespace vectorized::schema_util { +class LocalSchemaChangeRecorder; +} class BetaRowsetWriter : public RowsetWriter { public: @@ -80,6 +83,11 @@ class BetaRowsetWriter : public RowsetWriter { int32_t get_atomic_num_segment() const override { return _num_segment.load(); } + // Maybe modified by local schema change + vectorized::schema_util::LocalSchemaChangeRecorder* mutable_schema_change_recorder() { + return _context.schema_change_recorder.get(); + } + private: Status _add_block(const vectorized::Block* block, std::unique_ptr* writer); @@ -87,8 +95,10 @@ class BetaRowsetWriter : public RowsetWriter { std::unique_ptr* writer); Status _do_create_segment_writer(std::unique_ptr* writer, - bool is_segcompaction, int64_t begin, int64_t end); - Status _create_segment_writer(std::unique_ptr* writer); + bool is_segcompaction, int64_t begin, int64_t end, + const vectorized::Block* block = nullptr); + Status _create_segment_writer(std::unique_ptr* writer, + const vectorized::Block* block = nullptr); Status _create_segment_writer_for_segcompaction( std::unique_ptr* writer, uint64_t begin, uint64_t end); diff --git a/be/src/olap/rowset/rowset_meta.h b/be/src/olap/rowset/rowset_meta.h index 950323f15239b2..96970558643e37 100644 --- a/be/src/olap/rowset/rowset_meta.h +++ b/be/src/olap/rowset/rowset_meta.h @@ -361,7 +361,6 @@ class RowsetMeta { int64_t newest_write_timestamp() const { return _rowset_meta_pb.newest_write_timestamp(); } void set_tablet_schema(const TabletSchemaSPtr& tablet_schema) { - DCHECK(_schema == nullptr); _schema = TabletSchemaCache::instance()->insert(tablet_schema->to_key()); } diff --git a/be/src/olap/rowset/rowset_writer_context.h b/be/src/olap/rowset/rowset_writer_context.h index 8669a072de1b0a..3ab547d9cda840 100644 --- a/be/src/olap/rowset/rowset_writer_context.h +++ b/be/src/olap/rowset/rowset_writer_context.h @@ -26,6 +26,10 @@ namespace doris { class RowsetWriterContextBuilder; using RowsetWriterContextBuilderSharedPtr = std::shared_ptr; class DataDir; +class Tablet; +namespace vectorized::schema_util { +class LocalSchemaChangeRecorder; +} struct RowsetWriterContext { RowsetWriterContext() @@ -79,6 +83,10 @@ struct RowsetWriterContext { // If it is directly write from load procedure, else // it could be compaction or schema change etc.. bool is_direct_write = false; + std::shared_ptr tablet = nullptr; + // for tracing local schema change record + std::shared_ptr schema_change_recorder = + nullptr; }; } // 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 9a90cb314fc705..0b8d2d64772d3b 100644 --- a/be/src/olap/rowset/segment_v2/segment_writer.cpp +++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp @@ -34,6 +34,7 @@ #include "util/crc32c.h" #include "util/faststring.h" #include "util/key_util.h" +#include "vec/common/schema_util.h" namespace doris { namespace segment_v2 { @@ -88,22 +89,27 @@ void SegmentWriter::init_column_meta(ColumnMetaPB* meta, uint32_t column_id, } } -Status SegmentWriter::init() { +Status SegmentWriter::init(const vectorized::Block* block) { std::vector column_ids; - for (uint32_t i = 0; i < _tablet_schema->num_columns(); ++i) { + int column_cnt = _tablet_schema->num_columns(); + if (block) { + column_cnt = block->columns(); + } + for (uint32_t i = 0; i < column_cnt; ++i) { column_ids.emplace_back(i); } - return init(column_ids, true); + return init(column_ids, true, block); } -Status SegmentWriter::init(const std::vector& col_ids, bool has_key) { +Status SegmentWriter::init(const std::vector& col_ids, bool has_key, + const vectorized::Block* block) { DCHECK(_column_writers.empty()); DCHECK(_column_ids.empty()); _has_key = has_key; _column_writers.reserve(_tablet_schema->columns().size()); _column_ids.insert(_column_ids.end(), col_ids.begin(), col_ids.end()); - for (auto& cid : col_ids) { - const auto& column = _tablet_schema->column(cid); + _olap_data_convertor = std::make_unique(); + auto create_column_writer = [&](uint32_t cid, const auto& column) -> auto{ ColumnWriterOptions opts; opts.meta = _footer.add_columns(); @@ -182,6 +188,15 @@ Status SegmentWriter::init(const std::vector& col_ids, bool has_key) { RETURN_IF_ERROR(ColumnWriter::create(opts, &column, _file_writer, &writer)); RETURN_IF_ERROR(writer->init()); _column_writers.push_back(std::move(writer)); + + _olap_data_convertor->add_column_data_convertor(column); + return Status::OK(); + }; + + if (block) { + RETURN_IF_ERROR(_create_writers_with_block(block, create_column_writer)); + } else { + RETURN_IF_ERROR(_create_writers(create_column_writer)); } // we don't need the short key index for unique key merge on write table. @@ -200,9 +215,51 @@ Status SegmentWriter::init(const std::vector& col_ids, bool has_key) { new ShortKeyIndexBuilder(_segment_id, _opts.num_rows_per_block)); } } - // init olap data converter - _olap_data_convertor = - std::make_unique(_tablet_schema.get(), _column_ids); + return Status::OK(); +} + +Status SegmentWriter::_create_writers( + std::function create_column_writer) { + _olap_data_convertor->reserve(_column_ids.size()); + for (auto& cid : _column_ids) { + RETURN_IF_ERROR(create_column_writer(cid, _tablet_schema->column(cid))); + } + return Status::OK(); +} + +Status SegmentWriter::_create_writers_with_block( + const vectorized::Block* block, + std::function create_column_writer) { + // generate writers from schema and extended schema info + _olap_data_convertor->reserve(block->columns()); + // new columns added, query column info from Master + vectorized::schema_util::FullBaseSchemaView schema_view; + if (block->columns() > _tablet_schema->num_columns()) { + schema_view.table_id = _tablet_schema->table_id(); + RETURN_IF_ERROR( + vectorized::schema_util::send_fetch_full_base_schema_view_rpc(&schema_view)); + } + for (size_t i = 0; i < block->columns(); ++i) { + const auto& column_type_name = block->get_by_position(i); + auto idx = _tablet_schema->field_index(column_type_name.name); + if (idx >= 0) { + RETURN_IF_ERROR(create_column_writer(i, _tablet_schema->column(idx))); + } else { + if (schema_view.column_name_to_column.count(column_type_name.name) == 0) { + // expr columns, maybe happend in query like `insert into table1 select function(column1), column2 from table2` + // the first column name may become `function(column1)`, so we use column offset to get columns info + // TODO here we could optimize to col_unique_id in the future + RETURN_IF_ERROR(create_column_writer(i, _tablet_schema->column(i))); + continue; + } + // extended columns + const auto& tcolumn = schema_view.column_name_to_column[column_type_name.name]; + TabletColumn new_column(tcolumn); + RETURN_IF_ERROR(create_column_writer(i, new_column)); + _opts.rowset_ctx->schema_change_recorder->add_extended_columns( + new_column, schema_view.schema_version); + } + } return Status::OK(); } @@ -219,7 +276,7 @@ void SegmentWriter::_maybe_invalid_row_cache(const std::string& key) { Status SegmentWriter::append_block(const vectorized::Block* block, size_t row_pos, size_t num_rows) { - CHECK(block->columns() == _column_writers.size()) + CHECK(block->columns() >= _column_writers.size()) << ", block->columns()=" << block->columns() << ", _column_writers.size()=" << _column_writers.size(); diff --git a/be/src/olap/rowset/segment_v2/segment_writer.h b/be/src/olap/rowset/segment_v2/segment_writer.h index ebdec857f0e71f..1bba784f5888bc 100644 --- a/be/src/olap/rowset/segment_v2/segment_writer.h +++ b/be/src/olap/rowset/segment_v2/segment_writer.h @@ -73,10 +73,11 @@ class SegmentWriter { uint32_t max_row_per_segment, const SegmentWriterOptions& opts); ~SegmentWriter(); - Status init(); + Status init(const vectorized::Block* block = nullptr); // for vertical compaction - Status init(const std::vector& col_ids, bool has_key); + Status init(const std::vector& col_ids, bool has_key, + const vectorized::Block* block = nullptr); template Status append_row(const RowType& row); @@ -108,6 +109,10 @@ class SegmentWriter { bool is_unique_key() { return _tablet_schema->keys_type() == UNIQUE_KEYS; } private: + Status _create_writers_with_block( + const vectorized::Block* block, + std::function writer_creator); + Status _create_writers(std::function writer_creator); DISALLOW_COPY_AND_ASSIGN(SegmentWriter); Status _write_data(); Status _write_ordinal_index(); diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp index 25bbf438037023..61bb4bcf56f17d 100644 --- a/be/src/olap/tablet_meta.cpp +++ b/be/src/olap/tablet_meta.cpp @@ -228,6 +228,11 @@ TabletMeta::TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id if (tablet_schema.__isset.disable_auto_compaction) { schema->set_disable_auto_compaction(tablet_schema.disable_auto_compaction); } + + if (tablet_schema.__isset.is_dynamic_schema) { + schema->set_is_dynamic_schema(tablet_schema.is_dynamic_schema); + } + if (tablet_schema.__isset.delete_sign_idx) { schema->set_delete_sign_idx(tablet_schema.delete_sign_idx); } diff --git a/be/src/olap/tablet_schema.cpp b/be/src/olap/tablet_schema.cpp index 423d25eb5e212a..7ceee74f2aa613 100644 --- a/be/src/olap/tablet_schema.cpp +++ b/be/src/olap/tablet_schema.cpp @@ -655,6 +655,7 @@ void TabletSchema::init_from_pb(const TabletSchemaPB& schema) { _is_in_memory = schema.is_in_memory(); _disable_auto_compaction = schema.disable_auto_compaction(); _store_row_column = schema.store_row_column(); + _is_dynamic_schema = schema.is_dynamic_schema(); _delete_sign_idx = schema.delete_sign_idx(); _sequence_col_idx = schema.sequence_col_idx(); _sort_type = schema.sort_type(); @@ -791,6 +792,7 @@ void TabletSchema::to_schema_pb(TabletSchemaPB* tablet_schema_pb) const { tablet_schema_pb->set_sort_col_num(_sort_col_num); tablet_schema_pb->set_schema_version(_schema_version); tablet_schema_pb->set_compression_type(_compression_type); + tablet_schema_pb->set_is_dynamic_schema(_is_dynamic_schema); } size_t TabletSchema::row_size() const { diff --git a/be/src/olap/tablet_schema.h b/be/src/olap/tablet_schema.h index 0c154464534946..ff53949541684d 100644 --- a/be/src/olap/tablet_schema.h +++ b/be/src/olap/tablet_schema.h @@ -54,6 +54,7 @@ class TabletColumn { void set_type(FieldType type) { _type = type; } bool is_key() const { return _is_key; } bool is_nullable() const { return _is_nullable; } + bool is_variant_type() const { return _type == OLAP_FIELD_TYPE_VARIANT; } bool is_bf_column() const { return _is_bf_column; } bool has_bitmap_index() const { return _has_bitmap_index; } bool is_array_type() const { return _type == OLAP_FIELD_TYPE_ARRAY; } @@ -207,6 +208,7 @@ class TabletSchema { bool disable_auto_compaction() const { return _disable_auto_compaction; } void set_store_row_column(bool store_row_column) { _store_row_column = store_row_column; } bool store_row_column() const { return _store_row_column; } + bool is_dynamic_schema() const { return _is_dynamic_schema; } int32_t delete_sign_idx() const { return _delete_sign_idx; } void set_delete_sign_idx(int32_t delete_sign_idx) { _delete_sign_idx = delete_sign_idx; } bool has_sequence_col() const { return _sequence_col_idx != -1; } @@ -227,7 +229,10 @@ class TabletSchema { const std::vector& return_columns, const std::unordered_set* tablet_columns_need_convert_null = nullptr) const; vectorized::Block create_block(bool ignore_dropped_col = true) const; + void set_schema_version(int32_t version) { _schema_version = version; } + void set_table_id(int32_t table_id) { _table_id = table_id; } + int32_t table_id() const { return _table_id; } void build_current_tablet_schema(int64_t index_id, int32_t version, const OlapTableIndexSchema* index, const TabletSchema& out_tablet_schema); @@ -271,9 +276,11 @@ class TabletSchema { bool _has_bf_fpp = false; double _bf_fpp = 0; bool _is_in_memory = false; + bool _is_dynamic_schema = false; int32_t _delete_sign_idx = -1; int32_t _sequence_col_idx = -1; int32_t _schema_version = -1; + int32_t _table_id = -1; bool _disable_auto_compaction = false; int64_t _mem_size = 0; bool _store_row_column = false; diff --git a/be/src/olap/types.cpp b/be/src/olap/types.cpp index 0ccf4b98b168e9..f92c0c0da1acfa 100644 --- a/be/src/olap/types.cpp +++ b/be/src/olap/types.cpp @@ -33,6 +33,7 @@ bool is_scalar_type(FieldType field_type) { case OLAP_FIELD_TYPE_STRUCT: case OLAP_FIELD_TYPE_ARRAY: case OLAP_FIELD_TYPE_MAP: + case OLAP_FIELD_TYPE_VARIANT: return false; default: return true; diff --git a/be/src/runtime/define_primitive_type.h b/be/src/runtime/define_primitive_type.h index 5a87112398543d..8ce3da99f55f28 100644 --- a/be/src/runtime/define_primitive_type.h +++ b/be/src/runtime/define_primitive_type.h @@ -54,6 +54,7 @@ enum PrimitiveType { TYPE_DECIMAL64, /* 29 */ TYPE_DECIMAL128I, /* 30 */ TYPE_JSONB, /* 31 */ + TYPE_VARIANT /* 32 */ }; } diff --git a/be/src/runtime/descriptors.cpp b/be/src/runtime/descriptors.cpp index 5f392e700ae084..01ed59eaac178e 100644 --- a/be/src/runtime/descriptors.cpp +++ b/be/src/runtime/descriptors.cpp @@ -123,6 +123,7 @@ std::string SlotDescriptor::debug_string() const { TableDescriptor::TableDescriptor(const TTableDescriptor& tdesc) : _name(tdesc.tableName), _database(tdesc.dbName), + _table_id(tdesc.id), _num_cols(tdesc.numCols), _num_clustering_cols(tdesc.numClusteringCols) {} diff --git a/be/src/runtime/descriptors.h b/be/src/runtime/descriptors.h index c9fec4f933a0dc..fd75a5ad032987 100644 --- a/be/src/runtime/descriptors.h +++ b/be/src/runtime/descriptors.h @@ -178,10 +178,12 @@ class TableDescriptor { const std::string& name() const { return _name; } const std::string& database() const { return _database; } + int32_t table_id() const { return _table_id; } private: std::string _name; std::string _database; + int32_t _table_id; int _num_cols; int _num_clustering_cols; }; diff --git a/be/src/runtime/primitive_type.cpp b/be/src/runtime/primitive_type.cpp index f150e0165a72fd..7a90ffbcf2d234 100644 --- a/be/src/runtime/primitive_type.cpp +++ b/be/src/runtime/primitive_type.cpp @@ -550,6 +550,47 @@ TTypeDesc gen_type_desc(const TPrimitiveType::type val, const std::string& name) return type_desc; } +PrimitiveType get_primitive_type(vectorized::TypeIndex v_type) { + switch (v_type) { + case vectorized::TypeIndex::Int8: + return PrimitiveType::TYPE_TINYINT; + case vectorized::TypeIndex::Int16: + return PrimitiveType::TYPE_SMALLINT; + case vectorized::TypeIndex::Int32: + return PrimitiveType::TYPE_INT; + case vectorized::TypeIndex::Int64: + return PrimitiveType::TYPE_BIGINT; + case vectorized::TypeIndex::Float32: + return PrimitiveType::TYPE_FLOAT; + case vectorized::TypeIndex::Float64: + return PrimitiveType::TYPE_DOUBLE; + case vectorized::TypeIndex::Decimal32: + return PrimitiveType::TYPE_DECIMALV2; + case vectorized::TypeIndex::Array: + return PrimitiveType::TYPE_ARRAY; + case vectorized::TypeIndex::String: + return PrimitiveType::TYPE_STRING; + case vectorized::TypeIndex::Date: + return PrimitiveType::TYPE_DATE; + case vectorized::TypeIndex::DateTime: + return PrimitiveType::TYPE_DATETIME; + case vectorized::TypeIndex::Tuple: + return PrimitiveType::TYPE_STRUCT; + case vectorized::TypeIndex::Decimal128: + return PrimitiveType::TYPE_DECIMAL128I; + case vectorized::TypeIndex::JSONB: + return PrimitiveType::TYPE_JSONB; + case vectorized::TypeIndex::DateTimeV2: + return PrimitiveType::TYPE_DATETIMEV2; + case vectorized::TypeIndex::DateV2: + return PrimitiveType::TYPE_DATEV2; + // TODO add vectorized::more types + default: + LOG(FATAL) << "unknow data_type: " << getTypeName(v_type); + return PrimitiveType::INVALID_TYPE; + } +} + int get_slot_size(PrimitiveType type) { switch (type) { case TYPE_CHAR: @@ -561,6 +602,8 @@ int get_slot_size(PrimitiveType type) { return sizeof(StringRef); case TYPE_JSONB: return sizeof(JsonBinaryValue); + case TYPE_VARIANT: + return sizeof(StringRef); case TYPE_ARRAY: return sizeof(CollectionValue); case TYPE_MAP: diff --git a/be/src/runtime/primitive_type.h b/be/src/runtime/primitive_type.h index e3f3e8390013f2..f6dacd1858b01c 100644 --- a/be/src/runtime/primitive_type.h +++ b/be/src/runtime/primitive_type.h @@ -103,6 +103,8 @@ int get_slot_size(PrimitiveType type); bool is_type_compatible(PrimitiveType lhs, PrimitiveType rhs); +PrimitiveType get_primitive_type(vectorized::TypeIndex v_type); + TExprOpcode::type to_in_opcode(PrimitiveType t); PrimitiveType thrift_to_type(TPrimitiveType::type ttype); TPrimitiveType::type to_thrift(PrimitiveType ptype); diff --git a/be/src/runtime/types.cpp b/be/src/runtime/types.cpp index 53a087e9108592..333bf90a1400cc 100644 --- a/be/src/runtime/types.cpp +++ b/be/src/runtime/types.cpp @@ -78,6 +78,13 @@ TypeDescriptor::TypeDescriptor(const std::vector& types, int* idx) } break; } + case TTypeNodeType::VARIANT: { + DCHECK(!node.__isset.scalar_type); + // variant column must be the last column + DCHECK_EQ(*idx, types.size() - 1); + type = TYPE_VARIANT; + break; + } // case TTypeNodeType::STRUCT: // type = TYPE_STRUCT; // for (int i = 0; i < node.struct_fields.size(); ++i) { @@ -120,6 +127,8 @@ void TypeDescriptor::to_thrift(TTypeDesc* thrift_type) const { } else if (type == TYPE_MAP) { //TODO(xy): need to process children for map node.type = TTypeNodeType::MAP; + } else if (type == TYPE_VARIANT) { + node.type = TTypeNodeType::VARIANT; } else { DCHECK_EQ(type, TYPE_STRUCT); node.type = TTypeNodeType::STRUCT; @@ -186,6 +195,8 @@ void TypeDescriptor::to_protobuf(PTypeDesc* ptype) const { for (const TypeDescriptor& child : children) { child.to_protobuf(ptype); } + } else if (type == TYPE_VARIANT) { + node->set_type(TTypeNodeType::VARIANT); } } @@ -247,6 +258,9 @@ TypeDescriptor::TypeDescriptor(const google::protobuf::RepeatedPtrField"; return ss.str(); } + case TYPE_VARIANT: + ss << "VARIANT"; + return ss.str(); default: return type_to_string(type); } diff --git a/be/src/runtime/types.h b/be/src/runtime/types.h index 3a5f4f14dd3732..51a7408ac5370f 100644 --- a/be/src/runtime/types.h +++ b/be/src/runtime/types.h @@ -192,7 +192,8 @@ struct TypeDescriptor { } bool is_complex_type() const { - return type == TYPE_STRUCT || type == TYPE_ARRAY || type == TYPE_MAP; + return type == TYPE_STRUCT || type == TYPE_ARRAY || type == TYPE_MAP || + type == TYPE_VARIANT; } bool is_collection_type() const { return type == TYPE_ARRAY || type == TYPE_MAP; } @@ -201,6 +202,8 @@ struct TypeDescriptor { bool is_bitmap_type() const { return type == TYPE_OBJECT; } + bool is_variant_type() const { return type == TYPE_VARIANT; } + int get_slot_size() const { return ::doris::get_slot_size(type); } static inline int get_decimal_byte_size(int precision) { diff --git a/be/src/udf/udf.h b/be/src/udf/udf.h index 8b62898e45feb6..88b06c5bc110da 100644 --- a/be/src/udf/udf.h +++ b/be/src/udf/udf.h @@ -91,6 +91,7 @@ class FunctionContext { TYPE_DECIMAL64, TYPE_DECIMAL128I, TYPE_JSONB, + TYPE_VARIANT }; struct TypeDesc { diff --git a/be/src/vec/CMakeLists.txt b/be/src/vec/CMakeLists.txt index ede1bdfae47f28..897f86062bdf96 100644 --- a/be/src/vec/CMakeLists.txt +++ b/be/src/vec/CMakeLists.txt @@ -57,6 +57,11 @@ set(VEC_FILES columns/column_vector.cpp columns/column_map.cpp columns/columns_common.cpp + columns/column_object.cpp + json/json_parser.cpp + json/parse2column.cpp + json/path_in_data.cpp + common/schema_util.cpp common/demangle.cpp common/exception.cpp common/mremap.cpp @@ -92,12 +97,14 @@ set(VEC_FILES data_types/data_type_decimal.cpp data_types/data_type_map.cpp data_types/get_least_supertype.cpp + data_types/convert_field_to_type.cpp data_types/nested_utils.cpp data_types/data_type_date.cpp data_types/data_type_date_time.cpp data_types/data_type_time_v2.cpp data_types/data_type_jsonb.cpp data_types/data_type_time.cpp + data_types/data_type_object.cpp exec/vaggregation_node.cpp exec/varrow_scanner.cpp exec/vsort_node.cpp diff --git a/be/src/vec/columns/column.cpp b/be/src/vec/columns/column.cpp index 44521cc98c0625..6f8b2c4feeee20 100644 --- a/be/src/vec/columns/column.cpp +++ b/be/src/vec/columns/column.cpp @@ -81,4 +81,29 @@ bool is_column_const(const IColumn& column) { return check_column(column); } +ColumnPtr IColumn::create_with_offsets(const Offsets64& offsets, const Field& default_field, + size_t total_rows, size_t shift) const { + if (offsets.size() + shift != size()) { + LOG(FATAL) << fmt::format( + "Incompatible sizes of offsets ({}), shift ({}) and size of column {}", + offsets.size(), shift, size()); + } + auto res = clone_empty(); + res->reserve(total_rows); + ssize_t current_offset = -1; + for (size_t i = 0; i < offsets.size(); ++i) { + ssize_t offsets_diff = static_cast(offsets[i]) - current_offset; + current_offset = offsets[i]; + if (offsets_diff > 1) { + res->insert_many(default_field, offsets_diff - 1); + } + res->insert_from(*this, i + shift); + } + ssize_t offsets_diff = static_cast(total_rows) - current_offset; + if (offsets_diff > 1) { + res->insert_many(default_field, offsets_diff - 1); + } + return res; +} + } // namespace doris::vectorized diff --git a/be/src/vec/columns/column.h b/be/src/vec/columns/column.h index ef78f10468203f..4689e2d38ec52c 100644 --- a/be/src/vec/columns/column.h +++ b/be/src/vec/columns/column.h @@ -134,6 +134,10 @@ class IColumn : public COW { virtual void set_rowset_segment_id(std::pair rowset_segment_id) {} virtual std::pair get_rowset_segment_id() const { return {}; } + virtual TypeIndex get_data_type() const { + LOG(FATAL) << "Cannot get_data_type() column " << get_name(); + __builtin_unreachable(); + } /// Returns number of values in column. virtual size_t size() const = 0; @@ -397,7 +401,10 @@ class IColumn : public COW { /// Creates new column with values column[indexes[:limit]]. If limit is 0, all indexes are used. /// Indexes must be one of the ColumnUInt. For default implementation, see select_index_impl from ColumnsCommon.h - // virtual Ptr index(const IColumn & indexes, size_t limit) const = 0; + virtual Ptr index(const IColumn& indexes, size_t limit) const { + LOG(FATAL) << "column not support index"; + __builtin_unreachable(); + } /** Compares (*this)[n] and rhs[m]. Column rhs should have the same type. * Returns negative number, 0, or positive number (*this)[n] is less, equal, greater than rhs[m] respectively. @@ -448,6 +455,31 @@ class IColumn : public COW { LOG(FATAL) << "not support"; } + /// Appends one field multiple times. Can be optimized in inherited classes. + virtual void insert_many(const Field& field, size_t length) { + for (size_t i = 0; i < length; ++i) { + insert(field); + } + } + /// Returns indices of values in column, that not equal to default value of column. + virtual void get_indices_of_non_default_rows(Offsets64& indices, size_t from, + size_t limit) const { + LOG(FATAL) << "column not support get_indices_of_non_default_rows"; + __builtin_unreachable(); + } + + template + void get_indices_of_non_default_rows_impl(IColumn::Offsets64& indices, size_t from, + size_t limit) const; + + /// Returns column with @total_size elements. + /// In result column values from current column are at positions from @offsets. + /// Other values are filled by @default_value. + /// @shift means how much rows to skip from the beginning of current column. + /// Used to create full column from sparse. + virtual Ptr create_with_offsets(const Offsets64& offsets, const Field& default_field, + size_t total_rows, size_t shift) const; + /** Split column to smaller columns. Each value goes to column index, selected by corresponding element of 'selector'. * Selector must contain values from 0 to num_columns - 1. * For default implementation, see scatter_impl. @@ -512,6 +544,14 @@ class IColumn : public COW { return res; } + static MutablePtr mutate(Ptr ptr) { + MutablePtr res = ptr->shallow_mutate(); /// Now use_count is 2. + ptr.reset(); /// Reset use_count to 1. + res->for_each_subcolumn( + [](WrappedPtr& subcolumn) { subcolumn = std::move(*subcolumn).mutate(); }); + return res; + } + /** Some columns can contain another columns inside. * So, we have a tree of columns. But not all combinations are possible. * There are the following rules: diff --git a/be/src/vec/columns/column_array.cpp b/be/src/vec/columns/column_array.cpp index 4afa62e36d3a1f..10d61864cb7239 100644 --- a/be/src/vec/columns/column_array.cpp +++ b/be/src/vec/columns/column_array.cpp @@ -46,6 +46,39 @@ extern const int TOO_LARGE_ARRAY_SIZE; */ static constexpr size_t max_array_size_as_field = 1000000; +template +ColumnPtr ColumnArray::index_impl(const PaddedPODArray& indexes, size_t limit) const { + assert(limit <= indexes.size()); + if (limit == 0) { + return ColumnArray::create(data->clone_empty()); + } + /// Convert indexes to UInt64 in case of overflow. + auto nested_indexes_column = ColumnUInt64::create(); + PaddedPODArray& nested_indexes = nested_indexes_column->get_data(); + nested_indexes.reserve(get_offsets().back()); + auto res = ColumnArray::create(data->clone_empty()); + Offsets64& res_offsets = res->get_offsets(); + res_offsets.resize(limit); + size_t current_offset = 0; + for (size_t i = 0; i < limit; ++i) { + for (size_t j = 0; j < size_at(indexes[i]); ++j) { + nested_indexes.push_back(offset_at(indexes[i]) + j); + } + current_offset += size_at(indexes[i]); + res_offsets[i] = current_offset; + } + if (current_offset != 0) { + res->data = data->index(*nested_indexes_column, current_offset); + } + return res; +} + +ColumnPtr ColumnArray::index(const IColumn& indexes, size_t limit) const { + return select_index_impl(*this, indexes, limit); +} + +INSTANTIATE_INDEX_IMPL(ColumnArray) + ColumnArray::ColumnArray(MutableColumnPtr&& nested_column, MutableColumnPtr&& offsets_column) : data(std::move(nested_column)), offsets(std::move(offsets_column)) { const ColumnOffsets* offsets_concrete = typeid_cast(offsets.get()); diff --git a/be/src/vec/columns/column_array.h b/be/src/vec/columns/column_array.h index 824cca8b2389db..a5f396d5200c64 100644 --- a/be/src/vec/columns/column_array.h +++ b/be/src/vec/columns/column_array.h @@ -87,7 +87,7 @@ class ColumnArray final : public COWHelper { const char* get_family_name() const override { return "Array"; } bool is_column_array() const override { return true; } bool can_be_inside_nullable() const override { return true; } - TypeIndex get_data_type() const { return TypeIndex::Array; } + TypeIndex get_data_type() const override { return TypeIndex::Array; } MutableColumnPtr clone_resized(size_t size) const override; size_t size() const override; Field operator[](size_t n) const override; @@ -106,7 +106,8 @@ class ColumnArray final : public COWHelper { ColumnPtr filter(const Filter& filt, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation& perm, size_t limit) const override; //ColumnPtr index(const IColumn & indexes, size_t limit) const; - //template ColumnPtr index_impl(const PaddedPODArray & indexes, size_t limit) const; + template + ColumnPtr index_impl(const PaddedPODArray& indexes, size_t limit) const; [[noreturn]] int compare_at(size_t n, size_t m, const IColumn& rhs_, int nan_direction_hint) const override { LOG(FATAL) << "compare_at not implemented"; @@ -177,6 +178,22 @@ class ColumnArray final : public COWHelper { } Status filter_by_selector(const uint16_t* sel, size_t sel_size, IColumn* col_ptr) override; + size_t get_number_of_dimensions() const { + const auto* nested_array = check_and_get_column(*data); + if (!nested_array) { + return 1; + } + return 1 + + nested_array + ->get_number_of_dimensions(); /// Every modern C++ compiler optimizes tail recursion. + } + + void get_indices_of_non_default_rows(Offsets64& indices, size_t from, + size_t limit) const override { + return get_indices_of_non_default_rows_impl(indices, from, limit); + } + + ColumnPtr index(const IColumn& indexes, size_t limit) const override; private: WrappedPtr data; diff --git a/be/src/vec/columns/column_complex.h b/be/src/vec/columns/column_complex.h index be52a156c3c719..0bffb5c842fd3e 100644 --- a/be/src/vec/columns/column_complex.h +++ b/be/src/vec/columns/column_complex.h @@ -122,6 +122,18 @@ class ColumnComplexType final : public COWHelper> LOG(FATAL) << "get_permutation not implemented"; } + [[noreturn]] TypeIndex get_data_type() const override { + LOG(FATAL) << "ColumnComplexType get_data_type not implemeted"; + } + + void get_indices_of_non_default_rows(IColumn::Offsets64& indices, size_t from, + size_t limit) const override { + LOG(FATAL) << "get_indices_of_non_default_rows not implemented"; + } + [[noreturn]] ColumnPtr index(const IColumn& indexes, size_t limit) const override { + LOG(FATAL) << "index not implemented"; + } + void reserve(size_t n) override { data.reserve(n); } void resize(size_t n) override { data.resize(n); } diff --git a/be/src/vec/columns/column_const.cpp b/be/src/vec/columns/column_const.cpp index d2cb9ef7e7cffc..f207ff67b92a6c 100644 --- a/be/src/vec/columns/column_const.cpp +++ b/be/src/vec/columns/column_const.cpp @@ -163,4 +163,25 @@ void ColumnConst::get_permutation(bool /*reverse*/, size_t /*limit*/, int /*nan_ } } +void ColumnConst::get_indices_of_non_default_rows(Offsets64& indices, size_t from, + size_t limit) const { + if (!data->is_default_at(0)) { + size_t to = limit && from + limit < size() ? from + limit : size(); + indices.reserve(indices.size() + to - from); + for (size_t i = from; i < to; ++i) { + indices.push_back(i); + } + } +} + +ColumnPtr ColumnConst::index(const IColumn& indexes, size_t limit) const { + if (limit == 0) { + limit = indexes.size(); + } + if (indexes.size() < limit) { + LOG(FATAL) << "Size of indexes is less than required " << std::to_string(limit); + } + return ColumnConst::create(data, limit); +} + } // namespace doris::vectorized diff --git a/be/src/vec/columns/column_const.h b/be/src/vec/columns/column_const.h index 9637a0943f0799..094c6b37d2722d 100644 --- a/be/src/vec/columns/column_const.h +++ b/be/src/vec/columns/column_const.h @@ -66,6 +66,8 @@ class ColumnConst final : public COWHelper { StringRef get_data_at(size_t) const override { return data->get_data_at(0); } + TypeIndex get_data_type() const override { return data->get_data_type(); } + UInt64 get64(size_t) const override { return data->get64(0); } UInt64 get_uint(size_t) const override { return data->get_uint(0); } @@ -99,6 +101,11 @@ class ColumnConst final : public COWHelper { void pop_back(size_t n) override { s -= n; } + void get_indices_of_non_default_rows(Offsets64& indices, size_t from, + size_t limit) const override; + + ColumnPtr index(const IColumn& indexes, size_t limit) const override; + StringRef serialize_value_into_arena(size_t, Arena& arena, char const*& begin) const override { return data->serialize_value_into_arena(0, arena, begin); } diff --git a/be/src/vec/columns/column_decimal.cpp b/be/src/vec/columns/column_decimal.cpp index 13902b54a71c03..4bca086e5f54cb 100644 --- a/be/src/vec/columns/column_decimal.cpp +++ b/be/src/vec/columns/column_decimal.cpp @@ -22,6 +22,8 @@ #include "common/config.h" #include "util/simd/bits.h" +#include "vec/columns/column_impl.h" +#include "vec/columns/columns_common.h" #include "vec/common/arena.h" #include "vec/common/assert_cast.h" #include "vec/common/exception.h" @@ -439,6 +441,11 @@ Decimal128I ColumnDecimal::get_scale_multiplier() const { return common::exp10_i128(scale); } +template +ColumnPtr ColumnDecimal::index(const IColumn& indexes, size_t limit) const { + return select_index_impl(*this, indexes, limit); +} + template class ColumnDecimal; template class ColumnDecimal; template class ColumnDecimal; diff --git a/be/src/vec/columns/column_decimal.h b/be/src/vec/columns/column_decimal.h index a3960067cb2b86..e785e0df4220d6 100644 --- a/be/src/vec/columns/column_decimal.h +++ b/be/src/vec/columns/column_decimal.h @@ -189,11 +189,20 @@ class ColumnDecimal final : public COWHelper ColumnPtr index_impl(const PaddedPODArray& indexes, size_t limit) const; + void get_indices_of_non_default_rows(IColumn::Offsets64& indices, size_t from, + size_t limit) const override { + return this->template get_indices_of_non_default_rows_impl(indices, from, limit); + } + + ColumnPtr index(const IColumn& indexes, size_t limit) const override; + ColumnPtr replicate(const IColumn::Offsets& offsets) const override; void replicate(const uint32_t* counts, size_t target_size, IColumn& column, size_t begin = 0, int count_sz = -1) const override; + TypeIndex get_data_type() const override { return TypeId::value; } + void get_extremes(Field& min, Field& max) const override; MutableColumns scatter(IColumn::ColumnIndex num_columns, diff --git a/be/src/vec/columns/column_dictionary.h b/be/src/vec/columns/column_dictionary.h index eada23d57162fb..9ed997c43f0ac8 100644 --- a/be/src/vec/columns/column_dictionary.h +++ b/be/src/vec/columns/column_dictionary.h @@ -24,6 +24,7 @@ #include "vec/columns/column.h" #include "vec/columns/column_string.h" #include "vec/columns/predicate_column.h" +#include "vec/common/pod_array.h" #include "vec/common/string_ref.h" #include "vec/core/types.h" @@ -113,6 +114,10 @@ class ColumnDictionary final : public COWHelper> { void reserve(size_t n) override { _codes.reserve(n); } + [[noreturn]] TypeIndex get_data_type() const override { + LOG(FATAL) << "ColumnDictionary get_data_type not implemeted"; + } + const char* get_family_name() const override { return "ColumnDictionary"; } [[noreturn]] MutableColumnPtr clone_resized(size_t size) const override { @@ -154,6 +159,11 @@ class ColumnDictionary final : public COWHelper> { bool is_fixed_and_contiguous() const override { return true; } + void get_indices_of_non_default_rows(IColumn::Offsets64& indices, size_t from, + size_t limit) const override { + LOG(FATAL) << "get_indices_of_non_default_rows not supported in ColumnDictionary"; + } + size_t size_of_value_if_fixed() const override { return sizeof(T); } [[noreturn]] StringRef get_raw_data() const override { @@ -187,6 +197,10 @@ class ColumnDictionary final : public COWHelper> { LOG(FATAL) << "append_data_by_selector is not supported in ColumnDictionary!"; } + [[noreturn]] ColumnPtr index(const IColumn& indexes, size_t limit) const override { + LOG(FATAL) << "index not implemented"; + } + Status filter_by_selector(const uint16_t* sel, size_t sel_size, IColumn* col_ptr) override { auto* res_col = reinterpret_cast(col_ptr); StringRef strings[sel_size]; diff --git a/be/src/vec/columns/column_dummy.h b/be/src/vec/columns/column_dummy.h index 957b33318905e1..40bdd9cfd2aea6 100644 --- a/be/src/vec/columns/column_dummy.h +++ b/be/src/vec/columns/column_dummy.h @@ -143,6 +143,10 @@ class IColumnDummy : public IColumn { bool is_dummy() const override { return true; } + [[noreturn]] TypeIndex get_data_type() const override { + LOG(FATAL) << "IColumnDummy get_data_type not implemeted"; + } + void replace_column_data(const IColumn& rhs, size_t row, size_t self_row = 0) override { LOG(FATAL) << "should not call the method in column dummy"; } @@ -151,6 +155,17 @@ class IColumnDummy : public IColumn { LOG(FATAL) << "should not call the method in column dummy"; } + void get_indices_of_non_default_rows(Offsets64&, size_t, size_t) const override { + LOG(FATAL) << "should not call the method in column dummy"; + } + + ColumnPtr index(const IColumn& indexes, size_t limit) const override { + if (indexes.size() < limit) { + LOG(FATAL) << "Size of indexes is less than required."; + } + return clone_dummy(limit ? limit : s); + } + protected: size_t s; }; diff --git a/be/src/vec/columns/column_fixed_length_object.h b/be/src/vec/columns/column_fixed_length_object.h index d9c19a9edad4f5..aba83c5f1c9f54 100644 --- a/be/src/vec/columns/column_fixed_length_object.h +++ b/be/src/vec/columns/column_fixed_length_object.h @@ -157,6 +157,17 @@ class ColumnFixedLengthObject final : public COWHelper(*res).insert_from(*this, selector[i]); } +template +void IColumn::get_indices_of_non_default_rows_impl(IColumn::Offsets64& indices, size_t from, + size_t limit) const { + size_t to = limit && from + limit < size() ? from + limit : size(); + indices.reserve(indices.size() + to - from); + for (size_t i = from; i < to; ++i) { + if (!static_cast(*this).is_default_at(i)) { + indices.push_back(i); + } + } +} + } // namespace doris::vectorized diff --git a/be/src/vec/columns/column_map.h b/be/src/vec/columns/column_map.h index 4840ee4f69968f..500fc43f3a94a2 100644 --- a/be/src/vec/columns/column_map.h +++ b/be/src/vec/columns/column_map.h @@ -50,7 +50,7 @@ class ColumnMap final : public COWHelper { std::string get_name() const override; const char* get_family_name() const override { return "Map"; } - TypeIndex get_data_type() const { return TypeIndex::Map; } + TypeIndex get_data_type() const override { return TypeIndex::Map; } void for_each_subcolumn(ColumnCallback callback) override { callback(keys); diff --git a/be/src/vec/columns/column_nothing.h b/be/src/vec/columns/column_nothing.h index d3cb88574cb56e..16ab5d171ff49a 100644 --- a/be/src/vec/columns/column_nothing.h +++ b/be/src/vec/columns/column_nothing.h @@ -41,6 +41,8 @@ class ColumnNothing final : public COWHelper { bool structure_equals(const IColumn& rhs) const override { return typeid(rhs) == typeid(ColumnNothing); } + + TypeIndex get_data_type() const override { return TypeIndex::Nothing; } }; } // namespace doris::vectorized diff --git a/be/src/vec/columns/column_nullable.cpp b/be/src/vec/columns/column_nullable.cpp index 7233d2612f2621..5ec9f43571d8fb 100644 --- a/be/src/vec/columns/column_nullable.cpp +++ b/be/src/vec/columns/column_nullable.cpp @@ -620,4 +620,10 @@ ColumnPtr remove_nullable(const ColumnPtr& column) { return column; } +ColumnPtr ColumnNullable::index(const IColumn& indexes, size_t limit) const { + ColumnPtr indexed_data = get_nested_column().index(indexes, limit); + ColumnPtr indexed_null_map = get_null_map_column().index(indexes, limit); + return ColumnNullable::create(indexed_data, indexed_null_map); +} + } // namespace doris::vectorized diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h index 1da36feeac09a4..fc8e901c94b252 100644 --- a/be/src/vec/columns/column_nullable.h +++ b/be/src/vec/columns/column_nullable.h @@ -84,6 +84,8 @@ class ColumnNullable final : public COWHelper { UInt64 get64(size_t n) const override { return nested_column->get64(n); } StringRef get_data_at(size_t n) const override; + TypeIndex get_data_type() const override { return TypeIndex::Nullable; } + /// Will insert null value if pos=nullptr void insert_data(const char* pos, size_t length) override; @@ -322,6 +324,12 @@ class ColumnNullable final : public COWHelper { std::pair get_rowset_segment_id() const override { return nested_column->get_rowset_segment_id(); } + void get_indices_of_non_default_rows(Offsets64& indices, size_t from, + size_t limit) const override { + get_indices_of_non_default_rows_impl(indices, from, limit); + } + + ColumnPtr index(const IColumn& indexes, size_t limit) const override; private: // the two functions will not update `_need_update_has_null` diff --git a/be/src/vec/columns/column_object.cpp b/be/src/vec/columns/column_object.cpp new file mode 100644 index 00000000000000..2dcbbcd6159853 --- /dev/null +++ b/be/src/vec/columns/column_object.cpp @@ -0,0 +1,886 @@ +// 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. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/Columns/ColumnObject.cpp +// and modified by Doris + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace doris::vectorized { +namespace { + +DataTypePtr create_array_of_type(DataTypePtr type, size_t num_dimensions) { + for (size_t i = 0; i < num_dimensions; ++i) { + type = std::make_shared(std::move(type)); + } + return type; +} + +DataTypePtr getBaseTypeOfArray(const DataTypePtr& type) { + /// Get raw pointers to avoid extra copying of type pointers. + const DataTypeArray* last_array = nullptr; + const auto* current_type = type.get(); + while (const auto* type_array = typeid_cast(current_type)) { + current_type = type_array->get_nested_type().get(); + last_array = type_array; + } + return last_array ? last_array->get_nested_type() : type; +} + +size_t getNumberOfDimensions(const IDataType& type) { + if (const auto* type_array = typeid_cast(&type)) { + return type_array->get_number_of_dimensions(); + } + return 0; +} + +DataTypePtr get_data_type_by_column(const IColumn& column) { + auto idx = column.get_data_type(); + if (WhichDataType(idx).is_simple()) { + return DataTypeFactory::instance().get(String(getTypeName(idx))); + } + if (WhichDataType(idx).is_nothing()) { + return std::make_shared(); + } + if (const auto* column_array = check_and_get_column(&column)) { + return std::make_shared(get_data_type_by_column(column_array->get_data())); + } + if (const auto* column_nullable = check_and_get_column(&column)) { + return make_nullable(get_data_type_by_column(column_nullable->get_nested_column())); + } + // TODO add more types + assert(false); + return nullptr; +} + +/// Recreates column with default scalar values and keeps sizes of arrays. +ColumnPtr recreate_column_with_default_value(const ColumnPtr& column, + const DataTypePtr& scalar_type, + size_t num_dimensions) { + const auto* column_array = check_and_get_column(column.get()); + if (column_array && num_dimensions) { + return ColumnArray::create( + recreate_column_with_default_value(column_array->get_data_ptr(), scalar_type, + num_dimensions - 1), + IColumn::mutate(column_array->get_offsets_ptr())); + } + return create_array_of_type(scalar_type, num_dimensions) + ->create_column() + ->clone_resized(column->size()); +} + +Array create_empty_array_field(size_t num_dimensions) { + assert(num_dimensions != 0); + Array array; + Array* current_array = &array; + for (size_t i = 1; i < num_dimensions; ++i) { + current_array->push_back(Array()); + current_array = ¤t_array->back().get(); + } + return array; +} + +/// Replaces NULL fields to given field or empty array. +class FieldVisitorReplaceNull : public StaticVisitor { +public: + explicit FieldVisitorReplaceNull(const Field& replacement_, size_t num_dimensions_) + : replacement(replacement_), num_dimensions(num_dimensions_) {} + Field operator()(const Null&) const { + return num_dimensions ? create_empty_array_field(num_dimensions) : replacement; + } + Field operator()(const Array& x) const { + assert(num_dimensions > 0); + const size_t size = x.size(); + Array res(size); + for (size_t i = 0; i < size; ++i) { + res[i] = apply_visitor(FieldVisitorReplaceNull(replacement, num_dimensions - 1), x[i]); + } + return res; + } + template + Field operator()(const T& x) const { + return x; + } + +private: + const Field& replacement; + size_t num_dimensions; +}; + +/// Calculates number of dimensions in array field. +/// Returns 0 for scalar fields. +class FieldVisitorToNumberOfDimensions : public StaticVisitor { +public: + explicit FieldVisitorToNumberOfDimensions(Status* st) : _st(st) {} + size_t operator()(const Array& x) const { + const size_t size = x.size(); + std::optional dimensions; + for (size_t i = 0; i < size; ++i) { + /// Do not count Nulls, because they will be replaced by default + /// values with proper number of dimensions. + if (x[i].is_null()) { + continue; + } + size_t current_dimensions = apply_visitor(*this, x[i]); + if (!dimensions) { + dimensions = current_dimensions; + } else if (current_dimensions != *dimensions) { + *_st = Status::InvalidArgument( + "Number of dimensions mismatched among array elements"); + return 0; + } + } + return 1 + dimensions.value_or(0); + } + template + size_t operator()(const T&) const { + return 0; + } + +private: + mutable Status* _st; +}; + +/// Visitor that allows to get type of scalar field +/// or least common type of scalars in array. +/// More optimized version of FieldToDataType. +class FieldVisitorToScalarType : public StaticVisitor { +public: + using FieldType = Field::Types::Which; + size_t operator()(const Array& x) { + size_t size = x.size(); + for (size_t i = 0; i < size; ++i) { + apply_visitor(*this, x[i]); + } + return 0; + } + // TODO doris not support unsigned integers for now + // treat as signed integers + size_t operator()(const UInt64& x) { + field_types.insert(FieldType::UInt64); + if (x <= std::numeric_limits::max()) { + type_indexes.insert(TypeIndex::Int8); + } else if (x <= std::numeric_limits::max()) { + type_indexes.insert(TypeIndex::Int16); + } else if (x <= std::numeric_limits::max()) { + type_indexes.insert(TypeIndex::Int32); + } else { + type_indexes.insert(TypeIndex::Int64); + } + return 0; + } + size_t operator()(const Int64& x) { + // Only support Int32 and Int64 + field_types.insert(FieldType::Int64); + if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) { + type_indexes.insert(TypeIndex::Int32); + } else { + type_indexes.insert(TypeIndex::Int64); + } + return 0; + } + size_t operator()(const Null&) { + have_nulls = true; + return 0; + } + size_t operator()(const Int128I& x) { LOG(FATAL) << "not implemented"; } + template + size_t operator()(const T&) { + Field::EnumToType::Type a; + field_types.insert(Field::TypeToEnum>::value); + type_indexes.insert(TypeId>::value); + return 0; + } + Status get_scalar_type(DataTypePtr* type) const { + return get_least_supertype(type_indexes, type, true /*compatible with string type*/); + } + bool contain_nulls() const { return have_nulls; } + bool need_convert_field() const { return field_types.size() > 1; } + +private: + phmap::flat_hash_set type_indexes; + phmap::flat_hash_set field_types; + bool have_nulls = false; +}; + +} // namespace +Status get_field_info(const Field& field, FieldInfo* info) { + FieldVisitorToScalarType to_scalar_type_visitor; + apply_visitor(to_scalar_type_visitor, field); + DataTypePtr type = nullptr; + RETURN_IF_ERROR(to_scalar_type_visitor.get_scalar_type(&type)); + // array item's dimension may missmatch, eg. [1, 2, [1, 2, 3]] + Status num_to_dimensions_status; + *info = { + type, + to_scalar_type_visitor.contain_nulls(), + to_scalar_type_visitor.need_convert_field(), + apply_visitor(FieldVisitorToNumberOfDimensions(&num_to_dimensions_status), field), + }; + RETURN_IF_ERROR(num_to_dimensions_status); + return Status::OK(); +} + +ColumnObject::Subcolumn::Subcolumn(MutableColumnPtr&& data_, bool is_nullable_) + : least_common_type(get_data_type_by_column(*data_)), is_nullable(is_nullable_) { + data.push_back(std::move(data_)); +} + +ColumnObject::Subcolumn::Subcolumn(size_t size_, bool is_nullable_) + : least_common_type(std::make_shared()), + is_nullable(is_nullable_), + num_of_defaults_in_prefix(size_) {} + +size_t ColumnObject::Subcolumn::Subcolumn::size() const { + size_t res = num_of_defaults_in_prefix; + for (const auto& part : data) { + res += part->size(); + } + return res; +} + +size_t ColumnObject::Subcolumn::Subcolumn::byteSize() const { + size_t res = 0; + for (const auto& part : data) { + res += part->byte_size(); + } + return res; +} + +size_t ColumnObject::Subcolumn::Subcolumn::allocatedBytes() const { + size_t res = 0; + for (const auto& part : data) { + res += part->allocated_bytes(); + } + return res; +} + +Status ColumnObject::Subcolumn::insert(Field field) { + FieldInfo info; + RETURN_IF_ERROR(get_field_info(field, &info)); + return insert(std::move(field), std::move(info)); +} + +void ColumnObject::Subcolumn::add_new_column_part(DataTypePtr type) { + data.push_back(type->create_column()); + least_common_type = LeastCommonType {std::move(type)}; +} + +Status ColumnObject::Subcolumn::insert(Field field, FieldInfo info) { + auto base_type = std::move(info.scalar_type); + if (is_nothing(base_type)) { + insertDefault(); + return Status::OK(); + } + auto column_dim = least_common_type.get_dimensions(); + auto value_dim = info.num_dimensions; + if (is_nothing(least_common_type.getBase())) { + column_dim = value_dim; + } + if (is_nothing(base_type)) { + value_dim = column_dim; + } + if (value_dim != column_dim) { + return Status::InvalidArgument( + "Dimension of types mismatched between inserted value and column."); + } + if (is_nullable && !is_nothing(base_type)) { + base_type = make_nullable(base_type); + } + // alawys nullable at present + if (!is_nullable && info.have_nulls) { + field = apply_visitor(FieldVisitorReplaceNull(base_type->get_default(), value_dim), + std::move(field)); + } + // need replace muli dimensions array which contains null. eg. [[1, 2, 3], null] -> [[1, 2, 3], []] + // since column array doesnt known null's dimension + if (info.num_dimensions >= 2 && info.have_nulls) { + field = apply_visitor(FieldVisitorReplaceNull(base_type->get_default(), value_dim), + std::move(field)); + } + + bool type_changed = false; + const auto& least_common_base_type = least_common_type.getBase(); + if (data.empty()) { + add_new_column_part(create_array_of_type(std::move(base_type), value_dim)); + } else if (!least_common_base_type->equals(*base_type) && !is_nothing(base_type)) { + if (!schema_util::is_conversion_required_between_integers(*base_type, + *least_common_base_type)) { + RETURN_IF_ERROR( + get_least_supertype(DataTypes {std::move(base_type), least_common_base_type}, + &base_type, true /*compatible with string type*/)); + type_changed = true; + if (!least_common_base_type->equals(*base_type)) { + add_new_column_part(create_array_of_type(std::move(base_type), value_dim)); + } + } + } + + if (type_changed || info.need_convert) { + Field new_field; + RETURN_IF_ERROR(convert_field_to_type(field, *least_common_type.get(), &new_field)); + field = new_field; + } + + data.back()->insert(field); + return Status::OK(); +} + +Status ColumnObject::Subcolumn::insertRangeFrom(const Subcolumn& src, size_t start, size_t length) { + assert(src.is_finalized()); + const auto& src_column = src.data.back(); + const auto& src_type = src.least_common_type.get(); + if (data.empty()) { + add_new_column_part(src.least_common_type.get()); + data.back()->insert_range_from(*src_column, start, length); + } else if (least_common_type.get()->equals(*src_type)) { + data.back()->insert_range_from(*src_column, start, length); + } else { + DataTypePtr new_least_common_type = nullptr; + RETURN_IF_ERROR(get_least_supertype(DataTypes {least_common_type.get(), src_type}, + &new_least_common_type, + true /*compatible with string type*/)); + ColumnPtr casted_column; + RETURN_IF_ERROR(schema_util::cast_column({src_column, src_type, ""}, new_least_common_type, + &casted_column)); + if (!least_common_type.get()->equals(*new_least_common_type)) { + add_new_column_part(std::move(new_least_common_type)); + } + data.back()->insert_range_from(*casted_column, start, length); + } + return Status::OK(); +} + +bool ColumnObject::Subcolumn::is_finalized() const { + return data.empty() || (data.size() == 1 && num_of_defaults_in_prefix == 0); +} + +template +ColumnPtr ColumnObject::apply_for_subcolumns(Func&& func, std::string_view func_name) const { + if (!is_finalized()) { + LOG(FATAL) << "Cannot " << func_name << " non-finalized ColumnObject"; + } + auto res = ColumnObject::create(is_nullable); + for (const auto& subcolumn : subcolumns) { + auto new_subcolumn = func(subcolumn->data.get_finalized_column()); + res->add_sub_column(subcolumn->path, new_subcolumn->assume_mutable()); + } + return res; +} +ColumnPtr ColumnObject::index(const IColumn& indexes, size_t limit) const { + return apply_for_subcolumns( + [&](const auto& subcolumn) { return subcolumn.index(indexes, limit); }, "index"); +} + +void ColumnObject::Subcolumn::finalize() { + if (is_finalized()) { + return; + } + if (data.size() == 1 && num_of_defaults_in_prefix == 0) { + data[0] = data[0]->convert_to_full_column_if_const(); + return; + } + const auto& to_type = least_common_type.get(); + auto result_column = to_type->create_column(); + if (num_of_defaults_in_prefix) { + result_column->insert_many_defaults(num_of_defaults_in_prefix); + } + for (auto& part : data) { + part = part->convert_to_full_column_if_const(); + auto from_type = get_data_type_by_column(*part); + size_t part_size = part->size(); + if (!from_type->equals(*to_type)) { + auto offsets = ColumnUInt64::create(); + auto& offsets_data = offsets->get_data(); + /// We need to convert only non-default values and then recreate column + /// with default value of new type, because default values (which represents misses in data) + /// may be inconsistent between types (e.g "0" in UInt64 and empty string in String). + part->get_indices_of_non_default_rows(offsets_data, 0, part_size); + if (offsets->size() == part_size) { + ColumnPtr ptr; + schema_util::cast_column({part, from_type, ""}, to_type, &ptr); + part = ptr; + } else { + auto values = part->index(*offsets, offsets->size()); + schema_util::cast_column({values, from_type, ""}, to_type, &values); + part = values->create_with_offsets(offsets_data, to_type->get_default(), part_size, + /*shift=*/0); + } + } + result_column->insert_range_from(*part, 0, part_size); + } + data = {std::move(result_column)}; + num_of_defaults_in_prefix = 0; +} + +void ColumnObject::Subcolumn::insertDefault() { + if (data.empty()) { + ++num_of_defaults_in_prefix; + } else { + data.back()->insert_default(); + } +} + +void ColumnObject::Subcolumn::insertManyDefaults(size_t length) { + if (data.empty()) { + num_of_defaults_in_prefix += length; + } else { + data.back()->insert_many_defaults(length); + } +} + +void ColumnObject::Subcolumn::pop_back(size_t n) { + assert(n <= size()); + size_t num_removed = 0; + for (auto it = data.rbegin(); it != data.rend(); ++it) { + if (n == 0) { + break; + } + auto& column = *it; + if (n < column->size()) { + column->pop_back(n); + n = 0; + } else { + ++num_removed; + n -= column->size(); + } + } + data.resize(data.size() - num_removed); + num_of_defaults_in_prefix -= n; +} + +Field ColumnObject::Subcolumn::get_last_field() const { + if (data.empty()) { + return Field(); + } + const auto& last_part = data.back(); + assert(!last_part->empty()); + return (*last_part)[last_part->size() - 1]; +} + +ColumnObject::Subcolumn ColumnObject::Subcolumn::recreate_with_default_values( + const FieldInfo& field_info) const { + auto scalar_type = field_info.scalar_type; + if (is_nullable) { + scalar_type = make_nullable(scalar_type); + } + Subcolumn new_subcolumn; + new_subcolumn.least_common_type = + LeastCommonType {create_array_of_type(scalar_type, field_info.num_dimensions)}; + new_subcolumn.is_nullable = is_nullable; + new_subcolumn.num_of_defaults_in_prefix = num_of_defaults_in_prefix; + new_subcolumn.data.reserve(data.size()); + for (const auto& part : data) { + new_subcolumn.data.push_back( + recreate_column_with_default_value(part, scalar_type, field_info.num_dimensions)); + } + return new_subcolumn; +} + +IColumn& ColumnObject::Subcolumn::get_finalized_column() { + assert(is_finalized()); + return *data[0]; +} + +const IColumn& ColumnObject::Subcolumn::get_finalized_column() const { + assert(is_finalized()); + return *data[0]; +} + +const ColumnPtr& ColumnObject::Subcolumn::get_finalized_column_ptr() const { + assert(is_finalized()); + return data[0]; +} + +ColumnObject::Subcolumn::LeastCommonType::LeastCommonType(DataTypePtr type_) + : type(std::move(type_)), + base_type(getBaseTypeOfArray(type)), + num_dimensions(getNumberOfDimensions(*type)) {} + +ColumnObject::ColumnObject(bool is_nullable_) : is_nullable(is_nullable_), num_rows(0) {} + +ColumnObject::ColumnObject(Subcolumns&& subcolumns_, bool is_nullable_) + : is_nullable(is_nullable_), + subcolumns(std::move(subcolumns_)), + num_rows(subcolumns.empty() ? 0 : (*subcolumns.begin())->data.size()) { + check_consistency(); +} + +void ColumnObject::check_consistency() const { + if (subcolumns.empty()) { + return; + } + for (const auto& leaf : subcolumns) { + if (num_rows != leaf->data.size()) { + assert(false); + } + } +} + +size_t ColumnObject::size() const { +#ifndef NDEBUG + check_consistency(); +#endif + return num_rows; +} + +MutableColumnPtr ColumnObject::clone_resized(size_t new_size) const { + /// cloneResized with new_size == 0 is used for cloneEmpty(). + if (new_size != 0) { + LOG(FATAL) << "ColumnObject doesn't support resize to non-zero length"; + } + return ColumnObject::create(is_nullable); +} + +size_t ColumnObject::byte_size() const { + size_t res = 0; + for (const auto& entry : subcolumns) { + res += entry->data.byteSize(); + } + return res; +} + +size_t ColumnObject::allocated_bytes() const { + size_t res = 0; + for (const auto& entry : subcolumns) { + res += entry->data.allocatedBytes(); + } + return res; +} + +void ColumnObject::for_each_subcolumn(ColumnCallback callback) { + if (!is_finalized()) { + assert(false); + } + for (auto& entry : subcolumns) { + callback(entry->data.data.back()); + } +} + +Status ColumnObject::try_insert_from(const IColumn& src, size_t n) { + return try_insert(src[n]); +} + +Status ColumnObject::try_insert(const Field& field) { + const auto& object = field.get(); + phmap::flat_hash_set inserted; + size_t old_size = size(); + for (const auto& [key_str, value] : object) { + PathInData key(key_str); + inserted.insert(key_str); + if (!has_subcolumn(key)) { + bool succ = add_sub_column(key, old_size); + if (!succ) { + return Status::InvalidArgument( + fmt::format("Failed to add sub column {}", key.get_path())); + } + } + auto* subcolumn = get_subcolumn(key); + if (!subcolumn) { + return Status::InvalidArgument( + fmt::format("Failed to find sub column {}", key.get_path())); + } + RETURN_IF_ERROR(subcolumn->insert(value)); + } + for (auto& entry : subcolumns) { + if (!inserted.contains(entry->path.get_path())) { + entry->data.insertDefault(); + } + } + ++num_rows; + return Status::OK(); +} + +void ColumnObject::insert_default() { + for (auto& entry : subcolumns) { + entry->data.insertDefault(); + } + ++num_rows; +} + +Field ColumnObject::operator[](size_t n) const { + if (!is_finalized()) { + assert(false); + } + VariantMap map; + for (const auto& entry : subcolumns) { + map[entry->path.get_path()] = (*entry->data.data.back())[n]; + } + return map; +} + +void ColumnObject::get(size_t n, Field& res) const { + if (!is_finalized()) { + assert(false); + } + auto& map = res.get(); + for (const auto& entry : subcolumns) { + auto it = map.try_emplace(entry->path.get_path()).first; + entry->data.data.back()->get(n, it->second); + } +} + +Status ColumnObject::try_insert_indices_from(const IColumn& src, const int* indices_begin, + const int* indices_end) { + for (auto x = indices_begin; x != indices_end; ++x) { + if (*x == -1) { + ColumnObject::insert_default(); + } else { + RETURN_IF_ERROR(ColumnObject::try_insert_from(src, *x)); + } + } + finalize(); + return Status::OK(); +} + +Status ColumnObject::try_insert_range_from(const IColumn& src, size_t start, size_t length) { + const auto& src_object = assert_cast(src); + if (UNLIKELY(src_object.empty())) { + return Status::OK(); + } + for (auto& entry : subcolumns) { + if (src_object.has_subcolumn(entry->path)) { + auto* subcolumn = src_object.get_subcolumn(entry->path); + if (!subcolumn) { + return Status::InvalidArgument( + fmt::format("Failed to find sub column {}", entry->path.get_path())); + } + RETURN_IF_ERROR(entry->data.insertRangeFrom(*subcolumn, start, length)); + } else { + entry->data.insertManyDefaults(length); + } + } + for (const auto& entry : src_object.subcolumns) { + if (!has_subcolumn(entry->path)) { + bool succ = false; + if (entry->path.has_nested_part()) { + const auto& base_type = entry->data.get_least_common_typeBase(); + FieldInfo field_info { + .scalar_type = base_type, + .have_nulls = base_type->is_nullable(), + .need_convert = false, + .num_dimensions = entry->data.get_dimensions(), + }; + succ = add_nested_subcolumn(entry->path, field_info, num_rows); + } else { + succ = add_sub_column(entry->path, num_rows); + } + if (!succ) { + return Status::InvalidArgument( + fmt::format("Failed to add column {}", entry->path.get_path())); + } + auto* subcolumn = get_subcolumn(entry->path); + if (!subcolumn) { + return Status::InvalidArgument( + fmt::format("Failed to find sub column {}", entry->path.get_path())); + } + RETURN_IF_ERROR(subcolumn->insertRangeFrom(entry->data, start, length)); + } + } + num_rows += length; + finalize(); + return Status::OK(); +} + +void ColumnObject::pop_back(size_t length) { + for (auto& entry : subcolumns) { + entry->data.pop_back(length); + } + num_rows -= length; +} + +const ColumnObject::Subcolumn* ColumnObject::get_subcolumn(const PathInData& key) const { + const auto* node = subcolumns.find_leaf(key); + if (node == nullptr) { + VLOG_DEBUG << "There is no subcolumn " << key.get_path(); + return nullptr; + } + return &node->data; +} + +ColumnObject::Subcolumn* ColumnObject::get_subcolumn(const PathInData& key) { + const auto* node = subcolumns.find_leaf(key); + if (node == nullptr) { + VLOG_DEBUG << "There is no subcolumn " << key.get_path(); + return nullptr; + } + return &const_cast(node)->data; +} + +bool ColumnObject::has_subcolumn(const PathInData& key) const { + return subcolumns.find_leaf(key) != nullptr; +} + +bool ColumnObject::add_sub_column(const PathInData& key, MutableColumnPtr&& subcolumn) { + size_t new_size = subcolumn->size(); + bool inserted = subcolumns.add(key, Subcolumn(std::move(subcolumn), is_nullable)); + if (!inserted) { + VLOG_DEBUG << "Duplicated sub column " << key.get_path(); + return false; + } + if (num_rows == 0) { + num_rows = new_size; + } else if (new_size != num_rows) { + VLOG_DEBUG << "Size of subcolumn is in consistent with column"; + return false; + } + return true; +} + +bool ColumnObject::add_sub_column(const PathInData& key, size_t new_size) { + bool inserted = subcolumns.add(key, Subcolumn(new_size, is_nullable)); + if (!inserted) { + VLOG_DEBUG << "Duplicated sub column " << key.get_path(); + return false; + } + if (num_rows == 0) { + num_rows = new_size; + } else if (new_size != num_rows) { + VLOG_DEBUG << "Size of subcolumn is in consistent with column"; + return false; + } + return true; +} + +bool ColumnObject::add_nested_subcolumn(const PathInData& key, const FieldInfo& field_info, + size_t new_size) { + assert(key.has_nested_part()); + bool inserted = false; + /// We find node that represents the same Nested type as @key. + const auto* nested_node = subcolumns.find_best_match(key); + if (nested_node) { + /// Find any leaf of Nested subcolumn. + const auto* leaf = doris::vectorized::ColumnObject::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); + /// It's possible that we have already inserted value from current row + /// to this subcolumn. So, adjust size to expected. + if (new_subcolumn.size() > new_size) { + new_subcolumn.pop_back(new_subcolumn.size() - new_size); + } + assert(new_subcolumn.size() == new_size); + inserted = subcolumns.add(key, new_subcolumn); + } else { + /// If node was not found just add subcolumn with empty arrays. + inserted = subcolumns.add(key, Subcolumn(new_size, is_nullable)); + } + if (!inserted) { + VLOG_DEBUG << "Subcolumn already exists"; + return false; + } + if (num_rows == 0) { + num_rows = new_size; + } + return true; +} + +PathsInData ColumnObject::getKeys() const { + PathsInData keys; + keys.reserve(subcolumns.size()); + for (const auto& entry : subcolumns) { + keys.emplace_back(entry->path); + } + return keys; +} + +void ColumnObject::remove_subcolumns(const std::unordered_set& keys) { + Subcolumns new_subcolumns; + for (auto& entry : subcolumns) { + if (keys.count(entry->path.get_path()) == 0) { + new_subcolumns.add(entry->path, entry->data); + } + } + std::swap(subcolumns, new_subcolumns); +} + +bool ColumnObject::is_finalized() const { + return std::all_of(subcolumns.begin(), subcolumns.end(), + [](const auto& entry) { return entry->data.is_finalized(); }); +} + +void ColumnObject::finalize() { + size_t old_size = size(); + Subcolumns new_subcolumns; + for (auto&& entry : subcolumns) { + const auto& least_common_type = entry->data.get_least_common_type(); + /// Do not add subcolumns, which consists only from NULLs. + if (is_nothing(getBaseTypeOfArray(least_common_type))) { + continue; + } + entry->data.finalize(); + new_subcolumns.add(entry->path, entry->data); + } + /// If all subcolumns were skipped add a dummy subcolumn, + /// because Tuple type must have at least one element. + if (new_subcolumns.empty()) { + new_subcolumns.add( + PathInData {COLUMN_NAME_DUMMY}, + Subcolumn {static_cast(ColumnUInt8::create(old_size, 0)), + is_nullable}); + } + std::swap(subcolumns, new_subcolumns); +} + +bool ColumnObject::empty() const { + return subcolumns.empty() || subcolumns.begin()->get()->path.get_path() == COLUMN_NAME_DUMMY; +} + +ColumnPtr get_base_column_of_array(const ColumnPtr& column) { + if (const auto* column_array = check_and_get_column(column)) { + return column_array->get_data_ptr(); + } + return column; +} + +void ColumnObject::strip_outer_array() { + assert(is_finalized()); + size_t old_size = size(); + Subcolumns new_subcolumns; + for (auto&& entry : subcolumns) { + auto base_column = get_base_column_of_array(entry->data.get_finalized_column_ptr()); + new_subcolumns.add(entry->path, Subcolumn {base_column->assume_mutable(), is_nullable}); + num_rows = base_column->size(); + } + /// If all subcolumns were skipped add a dummy subcolumn, + /// because Tuple type must have at least one element. + if (new_subcolumns.empty()) { + new_subcolumns.add( + PathInData {COLUMN_NAME_DUMMY}, + Subcolumn {static_cast(ColumnUInt8::create(old_size, 0)), + is_nullable}); + } + std::swap(subcolumns, new_subcolumns); +} + +} // namespace doris::vectorized diff --git a/be/src/vec/columns/column_object.h b/be/src/vec/columns/column_object.h new file mode 100644 index 00000000000000..2bd804fc517b8d --- /dev/null +++ b/be/src/vec/columns/column_object.h @@ -0,0 +1,368 @@ +// 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. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/Columns/ColumnObject.h +// and modified by Doris + +#pragma once +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common/status.h" +namespace doris::vectorized { + +/// Info that represents a scalar or array field in a decomposed view. +/// It allows to recreate field with different number +/// of dimensions or nullability. +struct FieldInfo { + /// The common type of of all scalars in field. + DataTypePtr scalar_type; + /// Do we have NULL scalar in field. + bool have_nulls; + /// If true then we have scalars with different types in array and + /// we need to convert scalars to the common type. + bool need_convert; + /// Number of dimension in array. 0 if field is scalar. + size_t num_dimensions; +}; +Status get_field_info(const Field& field, FieldInfo* info); +/** A column that represents object with dynamic set of subcolumns. + * Subcolumns are identified by paths in document and are stored in + * a trie-like structure. ColumnObject is not suitable for writing into tables + * and it should be converted to Tuple with fixed set of subcolumns before that. + */ +class ColumnObject final : public COWHelper { +public: + /** Class that represents one subcolumn. + * It stores values in several parts of column + * and keeps current common type of all parts. + * We add a new column part with a new type, when we insert a field, + * which can't be converted to the current common type. + * After insertion of all values subcolumn should be finalized + * for writing and other operations. + */ + class Subcolumn { + public: + Subcolumn() = default; + + Subcolumn(size_t size_, bool is_nullable_); + + Subcolumn(MutableColumnPtr&& data_, bool is_nullable_); + + size_t size() const; + + size_t byteSize() const; + + size_t allocatedBytes() const; + + bool is_finalized() const; + + const DataTypePtr& get_least_common_type() const { return least_common_type.get(); } + + const DataTypePtr& get_least_common_typeBase() const { return least_common_type.getBase(); } + + size_t get_dimensions() const { return least_common_type.get_dimensions(); } + + /// Checks the consistency of column's parts stored in @data. + void checkTypes() const; + + /// Inserts a field, which scalars can be arbitrary, but number of + /// dimensions should be consistent with current common type. + /// return Status::InvalidArgument when meet conflict types + Status insert(Field field); + + Status insert(Field field, FieldInfo info); + + void insertDefault(); + + void insertManyDefaults(size_t length); + + Status insertRangeFrom(const Subcolumn& src, size_t start, size_t length); + + void pop_back(size_t n); + + /// Converts all column's parts to the common type and + /// creates a single column that stores all values. + void finalize(); + + /// Returns last inserted field. + Field get_last_field() const; + + /// 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; + + /// Returns single column if subcolumn in finalizes. + /// Otherwise -- undefined behaviour. + IColumn& get_finalized_column(); + + const IColumn& get_finalized_column() const; + + const ColumnPtr& get_finalized_column_ptr() const; + + friend class ColumnObject; + + private: + class LeastCommonType { + public: + LeastCommonType() = default; + + explicit LeastCommonType(DataTypePtr type_); + + const DataTypePtr& get() const { return type; } + + const DataTypePtr& getBase() const { return base_type; } + + size_t get_dimensions() const { return num_dimensions; } + + private: + DataTypePtr type; + DataTypePtr base_type; + size_t num_dimensions = 0; + }; + void add_new_column_part(DataTypePtr type); + + /// Current least common type of all values inserted to this subcolumn. + LeastCommonType least_common_type; + /// If true then common type type of subcolumn is Nullable + /// and default values are NULLs. + bool is_nullable = false; + /// Parts of column. Parts should be in increasing order in terms of subtypes/supertypes. + /// That means that the least common type for i-th prefix is the type of i-th part + /// and it's the supertype for all type of column from 0 to i-1. + std::vector data; + /// Until we insert any non-default field we don't know further + /// least common type and we count number of defaults in prefix, + /// which will be converted to the default type of final common type. + size_t num_of_defaults_in_prefix = 0; + }; + using Subcolumns = SubcolumnsTree; + +private: + /// If true then all subcolumns are nullable. + const bool is_nullable; + Subcolumns subcolumns; + size_t num_rows; + +public: + static constexpr auto COLUMN_NAME_DUMMY = "_dummy"; + + explicit ColumnObject(bool is_nullable_); + + ColumnObject(Subcolumns&& subcolumns_, bool is_nullable_); + + ~ColumnObject() override = default; + + bool can_be_inside_nullable() const override { return true; } + + /// Checks that all subcolumns have consistent sizes. + void check_consistency() const; + + bool has_subcolumn(const PathInData& key) const; + + // return null if not found + const Subcolumn* get_subcolumn(const PathInData& key) const; + + // return null if not found + Subcolumn* get_subcolumn(const PathInData& key); + + void incr_num_rows() { ++num_rows; } + + /// Adds a subcolumn from existing IColumn. + bool add_sub_column(const PathInData& key, MutableColumnPtr&& subcolumn); + + /// 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. + bool add_nested_subcolumn(const PathInData& key, const FieldInfo& field_info, size_t new_size); + + const Subcolumns& get_subcolumns() const { return subcolumns; } + + Subcolumns& get_subcolumns() { return subcolumns; } + + PathsInData getKeys() const; + + std::string get_keys_str() const { + std::stringstream ss; + bool first = true; + for (auto& k : getKeys()) { + if (first) { + first = false; + } else { + ss << ", "; + } + ss << k.get_path(); + } + + return ss.str(); + } + + void remove_subcolumns(const std::unordered_set& keys); + + /// Finalizes all subcolumns. + void finalize(); + + bool is_finalized() const; + + /// Part of interface + const char* get_family_name() const override { return "Variant"; } + + TypeIndex get_data_type() const override { return TypeIndex::VARIANT; } + + size_t size() const override; + + MutableColumnPtr clone_resized(size_t new_size) const override; + + size_t byte_size() const override; + + size_t allocated_bytes() const override; + + void for_each_subcolumn(ColumnCallback callback) override; + + // Do nothing, call try_insert instead + void insert(const Field& field) override { + Status st = try_insert(field); + if (!st.ok()) { + LOG(FATAL) << "insert return ERROR status: " << st; + } + } + + // Do nothing, call try_insert_range_from instead + void insert_range_from(const IColumn& src, size_t start, size_t length) override { + Status st = try_insert_range_from(src, start, length); + if (!st.ok()) { + LOG(FATAL) << "insert_range_from return ERROR status: " << st; + } + } + + // Only called in Block::add_row + Status try_insert(const Field& field); + + Status try_insert_from(const IColumn& src, size_t n); + + // Only called in Block::add_row + Status try_insert_range_from(const IColumn& src, size_t start, size_t length); + + void insert_default() override; + + [[noreturn]] ColumnPtr replicate(const Offsets& offsets) const override { + LOG(FATAL) << "should not call the method replicate in column object"; + } + + void pop_back(size_t length) override; + + Field operator[](size_t n) const override; + + void get(size_t n, Field& res) const override; + + /// All other methods throw exception. + StringRef get_data_at(size_t) const override { + LOG(FATAL) << "should not call the method in column object"; + return StringRef(); + } + + void insert_indices_from(const IColumn& src, const int* indices_begin, + const int* indices_end) override { + LOG(FATAL) << "should not call the method in column object"; + } + + Status try_insert_indices_from(const IColumn& src, const int* indices_begin, + const int* indices_end); + + StringRef serialize_value_into_arena(size_t n, Arena& arena, + char const*& begin) const override { + LOG(FATAL) << "should not call the method in column object"; + return StringRef(); + } + + const char* deserialize_and_insert_from_arena(const char* pos) override { + LOG(FATAL) << "should not call the method in column object"; + return nullptr; + } + + void update_hash_with_value(size_t n, SipHash& hash) const override { + LOG(FATAL) << "should not call the method in column object"; + } + + void insert_data(const char* pos, size_t length) override { + LOG(FATAL) << "should not call the method in column object"; + } + + ColumnPtr filter(const Filter&, ssize_t) const override { + LOG(FATAL) << "should not call the method in column object"; + return nullptr; + } + + ColumnPtr permute(const Permutation&, size_t) const override { + LOG(FATAL) << "should not call the method in column object"; + return nullptr; + } + + int compare_at(size_t n, size_t m, const IColumn& rhs, int nan_direction_hint) const override { + LOG(FATAL) << "should not call the method in column object"; + return 0; + } + + void get_permutation(bool reverse, size_t limit, int nan_direction_hint, + Permutation& res) const override { + LOG(FATAL) << "should not call the method in column object"; + } + + MutableColumns scatter(ColumnIndex, const Selector&) const override { + LOG(FATAL) << "should not call the method in column object"; + return {}; + } + + void replace_column_data(const IColumn&, size_t row, size_t self_row) override { + LOG(FATAL) << "should not call the method in column object"; + } + + void replace_column_data_default(size_t self_row) override { + LOG(FATAL) << "should not call the method in column object"; + } + + void get_extremes(Field& min, Field& max) const override { + LOG(FATAL) << "should not call the method in column object"; + } + + void get_indices_of_non_default_rows(Offsets64&, size_t, size_t) const override { + LOG(FATAL) << "should not call the method in column object"; + } + + void append_data_by_selector(MutableColumnPtr& res, + const IColumn::Selector& selector) const override { + LOG(FATAL) << "should not call the method in column object"; + } + + template + ColumnPtr apply_for_subcolumns(Func&& func, std::string_view func_name) const; + + ColumnPtr index(const IColumn& indexes, size_t limit) const override; + + void strip_outer_array(); + + bool empty() const; +}; +} // namespace doris::vectorized diff --git a/be/src/vec/columns/column_set.h b/be/src/vec/columns/column_set.h index 3b330f82284838..531946f2b6e669 100644 --- a/be/src/vec/columns/column_set.h +++ b/be/src/vec/columns/column_set.h @@ -43,6 +43,8 @@ class ColumnSet final : public COWHelper { ConstSetPtr get_data() const { return data; } + TypeIndex get_data_type() const override { return TypeIndex::String; } + private: ConstSetPtr data; }; diff --git a/be/src/vec/columns/column_string.cpp b/be/src/vec/columns/column_string.cpp index 0c0447c3417059..8c6ca64efe6ded 100644 --- a/be/src/vec/columns/column_string.cpp +++ b/be/src/vec/columns/column_string.cpp @@ -20,6 +20,7 @@ #include "vec/columns/column_string.h" +#include "vec/columns/column_impl.h" #include "vec/columns/columns_common.h" #include "vec/common/arena.h" #include "vec/common/assert_cast.h" @@ -520,4 +521,8 @@ void ColumnString::compare_internal(size_t rhs_row_id, const IColumn& rhs, int n } } +ColumnPtr ColumnString::index(const IColumn& indexes, size_t limit) const { + return select_index_impl(*this, indexes, limit); +} + } // namespace doris::vectorized diff --git a/be/src/vec/columns/column_string.h b/be/src/vec/columns/column_string.h index 920f480484f80d..710cba9e3e7b05 100644 --- a/be/src/vec/columns/column_string.h +++ b/be/src/vec/columns/column_string.h @@ -472,6 +472,24 @@ class ColumnString final : public COWHelper { void compare_internal(size_t rhs_row_id, const IColumn& rhs, int nan_direction_hint, int direction, std::vector& cmp_res, uint8* __restrict filter) const override; + MutableColumnPtr get_shinked_column() const { + auto shrinked_column = ColumnString::create(); + for (int i = 0; i < size(); i++) { + StringRef str = get_data_at(i); + reinterpret_cast(shrinked_column.get()) + ->insert_data(str.data, strnlen(str.data, str.size)); + } + return shrinked_column; + } + + TypeIndex get_data_type() const override { return TypeIndex::String; } + + void get_indices_of_non_default_rows(Offsets64& indices, size_t from, + size_t limit) const override { + return get_indices_of_non_default_rows_impl(indices, from, limit); + } + + ColumnPtr index(const IColumn& indexes, size_t limit) const override; }; } // namespace doris::vectorized diff --git a/be/src/vec/columns/column_struct.h b/be/src/vec/columns/column_struct.h index 627a6736f2e614..60d0e463154f34 100644 --- a/be/src/vec/columns/column_struct.h +++ b/be/src/vec/columns/column_struct.h @@ -99,7 +99,7 @@ class ColumnStruct final : public COWHelper { std::string get_name() const override; const char* get_family_name() const override { return "Struct"; } - TypeIndex get_data_type() const { return TypeIndex::Struct; } + TypeIndex get_data_type() const override { return TypeIndex::Struct; } bool can_be_inside_nullable() const override { return true; } MutableColumnPtr clone_empty() const override; MutableColumnPtr clone_resized(size_t size) const override; diff --git a/be/src/vec/columns/column_vector.cpp b/be/src/vec/columns/column_vector.cpp index 8e4bcfb5df2b64..ea5c14c0a82b11 100644 --- a/be/src/vec/columns/column_vector.cpp +++ b/be/src/vec/columns/column_vector.cpp @@ -28,6 +28,8 @@ #include "util/simd/bits.h" #include "util/stack_util.h" +#include "vec/columns/column_impl.h" +#include "vec/columns/columns_common.h" #include "vec/common/arena.h" #include "vec/common/assert_cast.h" #include "vec/common/bit_cast.h" @@ -547,6 +549,11 @@ void ColumnVector::get_extremes(Field& min, Field& max) const { max = NearestFieldType(cur_max); } +template +ColumnPtr ColumnVector::index(const IColumn& indexes, size_t limit) const { + return select_index_impl(*this, indexes, limit); +} + /// Explicit template instantiations - to avoid code bloat in headers. template class ColumnVector; template class ColumnVector; diff --git a/be/src/vec/columns/column_vector.h b/be/src/vec/columns/column_vector.h index c944827fbc832a..cd56cbdd044a9a 100644 --- a/be/src/vec/columns/column_vector.h +++ b/be/src/vec/columns/column_vector.h @@ -395,6 +395,13 @@ class ColumnVector final : public COWHelper> void compare_internal(size_t rhs_row_id, const IColumn& rhs, int nan_direction_hint, int direction, std::vector& cmp_res, uint8* __restrict filter) const override; + TypeIndex get_data_type() const override { return TypeId::value; } + void get_indices_of_non_default_rows(IColumn::Offsets64& indices, size_t from, + size_t limit) const override { + return this->template get_indices_of_non_default_rows_impl(indices, from, limit); + } + + ColumnPtr index(const IColumn& indexes, size_t limit) const override; protected: Container data; diff --git a/be/src/vec/columns/columns_common.cpp b/be/src/vec/columns/columns_common.cpp index e7af42e7e9f408..1988695eaa8ffe 100644 --- a/be/src/vec/columns/columns_common.cpp +++ b/be/src/vec/columns/columns_common.cpp @@ -279,4 +279,21 @@ INSTANTIATE(Float64, ColumnArray::Offset64) #undef INSTANTIATE +namespace detail { +template +const PaddedPODArray* get_indexes_data(const IColumn& indexes) { + auto* column = typeid_cast*>(&indexes); + if (column) { + return &column->get_data(); + } + + return nullptr; +} + +template const PaddedPODArray* get_indexes_data(const IColumn& indexes); +template const PaddedPODArray* get_indexes_data(const IColumn& indexes); +template const PaddedPODArray* get_indexes_data(const IColumn& indexes); +template const PaddedPODArray* get_indexes_data(const IColumn& indexes); +} // namespace detail + } // namespace doris::vectorized diff --git a/be/src/vec/columns/columns_common.h b/be/src/vec/columns/columns_common.h index 7308816005c95b..10233c6111886c 100644 --- a/be/src/vec/columns/columns_common.h +++ b/be/src/vec/columns/columns_common.h @@ -51,4 +51,45 @@ void filter_arrays_impl_only_data(const PaddedPODArray& src_elems, PaddedPODArray& res_elems, const IColumn::Filter& filt, ssize_t result_size_hint); +namespace detail { +template +const PaddedPODArray* get_indexes_data(const IColumn& indexes); +} + +/// Check limit <= indexes->size() and call column.index_impl(const PaddedPodArray & indexes, UInt64 limit). +template +ColumnPtr select_index_impl(const Column& column, const IColumn& indexes, size_t limit) { + if (limit == 0) { + limit = indexes.size(); + } + + if (indexes.size() < limit) { + LOG(FATAL) << "Size of indexes is less than required."; + } + + if (auto* data_uint8 = detail::get_indexes_data(indexes)) { + return column.template index_impl(*data_uint8, limit); + } else if (auto* data_uint16 = detail::get_indexes_data(indexes)) { + return column.template index_impl(*data_uint16, limit); + } else if (auto* data_uint32 = detail::get_indexes_data(indexes)) { + return column.template index_impl(*data_uint32, limit); + } else if (auto* data_uint64 = detail::get_indexes_data(indexes)) { + return column.template index_impl(*data_uint64, limit); + } else { + LOG(FATAL) << "Indexes column for IColumn::select must be ColumnUInt, got" + << indexes.get_name(); + return nullptr; + } +} + +#define INSTANTIATE_INDEX_IMPL(Column) \ + template ColumnPtr Column::index_impl(const PaddedPODArray& indexes, \ + size_t limit) const; \ + template ColumnPtr Column::index_impl(const PaddedPODArray& indexes, \ + size_t limit) const; \ + template ColumnPtr Column::index_impl(const PaddedPODArray& indexes, \ + size_t limit) const; \ + template ColumnPtr Column::index_impl(const PaddedPODArray& indexes, \ + size_t limit) const; + } // namespace doris::vectorized diff --git a/be/src/vec/columns/predicate_column.h b/be/src/vec/columns/predicate_column.h index 5a276233da5bef..2b00fee50c257a 100644 --- a/be/src/vec/columns/predicate_column.h +++ b/be/src/vec/columns/predicate_column.h @@ -186,6 +186,15 @@ class PredicateColumnType final : public COWHelper) { insert_string_to_res_column(sel, sel_size, diff --git a/be/src/vec/columns/subcolumn_tree.h b/be/src/vec/columns/subcolumn_tree.h new file mode 100644 index 00000000000000..f3de45e054e1a7 --- /dev/null +++ b/be/src/vec/columns/subcolumn_tree.h @@ -0,0 +1,221 @@ +// 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. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/DataTypes/Serializations/SubcolumnsTree.h +// and modified by Doris + +#pragma once +#include +#include +#include +#include +namespace doris::vectorized { +/// Tree that represents paths in document +/// with additional data in nodes. + +template +class SubcolumnsTree { +public: + struct Node { + enum Kind { + TUPLE, + NESTED, + SCALAR, + }; + + explicit Node(Kind kind_) : kind(kind_) {} + Node(Kind kind_, const NodeData& data_) : kind(kind_), data(data_) {} + Node(Kind kind_, const NodeData& data_, const PathInData& path_) + : kind(kind_), data(data_), path(path_) {} + + Kind kind = TUPLE; + const Node* parent = nullptr; + + Arena strings_pool; + HashMapWithStackMemory, StringRefHash, 4> children; + + NodeData data; + PathInData path; + + bool is_nested() const { return kind == NESTED; } + bool is_scalar() const { return kind == SCALAR; } + + void add_child(std::string_view key, std::shared_ptr next_node) { + next_node->parent = this; + StringRef key_ref {strings_pool.insert(key.data(), key.length()), key.length()}; + children[key_ref] = std::move(next_node); + } + }; + + using NodeKind = typename Node::Kind; + using NodePtr = std::shared_ptr; + + /// Add a leaf without any data in other nodes. + bool add(const PathInData& path, const NodeData& leaf_data) { + return add(path, [&](NodeKind kind, bool exists) -> NodePtr { + if (exists) { + return nullptr; + } + + if (kind == Node::SCALAR) { + return std::make_shared(kind, leaf_data, path); + } + + return std::make_shared(kind); + }); + } + + /// Callback for creation of node. Receives kind of node and + /// flag, which is true if node already exists. + using NodeCreator = std::function; + + bool add(const PathInData& path, const NodeCreator& node_creator) { + const auto& parts = path.get_parts(); + + if (parts.empty()) { + return false; + } + + if (!root) { + root = std::make_shared(Node::TUPLE); + } + + Node* current_node = root.get(); + for (size_t i = 0; i < parts.size() - 1; ++i) { + assert(current_node->kind != Node::SCALAR); + + auto it = current_node->children.find( + StringRef {parts[i].key.data(), parts[i].key.size()}); + if (it != current_node->children.end()) { + current_node = it->get_second().get(); + node_creator(current_node->kind, true); + + if (current_node->is_nested() != parts[i].is_nested) { + return false; + } + } else { + auto next_kind = parts[i].is_nested ? Node::NESTED : Node::TUPLE; + auto next_node = node_creator(next_kind, false); + current_node->add_child(String(parts[i].key), next_node); + current_node = next_node.get(); + } + } + + auto it = current_node->children.find( + StringRef {parts.back().key.data(), parts.back().key.size()}); + if (it != current_node->children.end()) { + return false; + } + + auto next_node = node_creator(Node::SCALAR, false); + current_node->add_child(String(parts.back().key), next_node); + leaves.push_back(std::move(next_node)); + + return true; + } + + /// Find node that matches the path the best. + const Node* find_best_match(const PathInData& path) const { return find_impl(path, false); } + + /// Find node that matches the path exactly. + const Node* find_exact(const PathInData& path) const { return find_impl(path, true); } + + /// Find leaf by path. + const Node* find_leaf(const PathInData& path) const { + const auto* candidate = find_exact(path); + if (!candidate || !candidate->is_scalar()) { + return nullptr; + } + return candidate; + } + + using NodePredicate = std::function; + + /// Finds leaf that satisfies the predicate. + const Node* find_leaf(const NodePredicate& predicate) { + return find_leaf(root.get(), predicate); + } + + static const Node* find_leaf(const Node* node, const NodePredicate& predicate) { + if (!node) { + return nullptr; + } + + if (node->is_scalar()) { + return predicate(*node) ? node : nullptr; + } + + for (auto it = node->children.begin(); it != node->children.end(); ++it) { + auto child = it->get_second(); + if (const auto* leaf = find_leaf(child.get(), predicate)) { + return leaf; + } + } + return nullptr; + } + + /// Find first parent node that satisfies the predicate. + static const Node* find_parent(const Node* node, const NodePredicate& predicate) { + while (node && !predicate(*node)) { + node = node->parent; + } + return node; + } + + bool empty() const { return root == nullptr; } + size_t size() const { return leaves.size(); } + + using Nodes = std::vector; + + const Nodes& get_leaves() const { return leaves; } + const Node* get_root() const { return root.get(); } + + using iterator = typename Nodes::iterator; + using const_iterator = typename Nodes::const_iterator; + + iterator begin() { return leaves.begin(); } + iterator end() { return leaves.end(); } + + const_iterator begin() const { return leaves.begin(); } + const_iterator end() const { return leaves.end(); } + +private: + const Node* find_impl(const PathInData& path, bool find_exact) const { + if (!root) { + return nullptr; + } + + const auto& parts = path.get_parts(); + const Node* current_node = root.get(); + + for (const auto& part : parts) { + auto it = current_node->children.find(StringRef {part.key.data(), part.key.size()}); + if (it == current_node->children.end()) { + return find_exact ? nullptr : current_node; + } + + current_node = it->get_second().get(); + } + + return current_node; + } + + NodePtr root; + Nodes leaves; +}; + +} // namespace doris::vectorized diff --git a/be/src/vec/common/hash_table/hash_map.h b/be/src/vec/common/hash_table/hash_map.h index 118b87087d9714..813950533c7fae 100644 --- a/be/src/vec/common/hash_table/hash_map.h +++ b/be/src/vec/common/hash_table/hash_map.h @@ -241,3 +241,10 @@ template , typename Grower = HashTableGrower<>, typename Allocator = HashTableAllocator> using HashMapWithSavedHash = HashMapTable, Hash, Grower, Allocator>; + +template +using HashMapWithStackMemory = HashMapTable< + Key, HashMapCellWithSavedHash, Hash, + HashTableGrower, + HashTableAllocatorWithStackMemory<(1ULL << initial_size_degree) * + sizeof(HashMapCellWithSavedHash)>>; \ No newline at end of file diff --git a/be/src/vec/common/schema_util.cpp b/be/src/vec/common/schema_util.cpp new file mode 100644 index 00000000000000..fe3d4ba475661a --- /dev/null +++ b/be/src/vec/common/schema_util.cpp @@ -0,0 +1,481 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include "gen_cpp/FrontendService.h" +#include "gen_cpp/HeartbeatService_types.h" +#include "olap/rowset/rowset_writer_context.h" +#include "runtime/client_cache.h" +#include "runtime/exec_env.h" +#include "util/thrift_rpc_helper.h" + +namespace doris::vectorized::schema_util { + +size_t get_number_of_dimensions(const IDataType& type) { + if (const auto* type_array = typeid_cast(&type)) { + return type_array->get_number_of_dimensions(); + } + return 0; +} +size_t get_number_of_dimensions(const IColumn& column) { + if (const auto* column_array = check_and_get_column(column)) { + return column_array->get_number_of_dimensions(); + } + return 0; +} + +DataTypePtr get_base_type_of_array(const DataTypePtr& type) { + /// Get raw pointers to avoid extra copying of type pointers. + const DataTypeArray* last_array = nullptr; + const auto* current_type = type.get(); + while (const auto* type_array = typeid_cast(current_type)) { + current_type = type_array->get_nested_type().get(); + last_array = type_array; + } + return last_array ? last_array->get_nested_type() : type; +} + +Array create_empty_array_field(size_t num_dimensions) { + DCHECK(num_dimensions > 0); + Array array; + Array* current_array = &array; + for (size_t i = 1; i < num_dimensions; ++i) { + current_array->push_back(Array()); + current_array = ¤t_array->back().get(); + } + return array; +} + +FieldType get_field_type(const IDataType* data_type) { + switch (data_type->get_type_id()) { + case TypeIndex::UInt8: + return FieldType::OLAP_FIELD_TYPE_UNSIGNED_TINYINT; + case TypeIndex::UInt16: + return FieldType::OLAP_FIELD_TYPE_UNSIGNED_SMALLINT; + case TypeIndex::UInt32: + return FieldType::OLAP_FIELD_TYPE_UNSIGNED_INT; + case TypeIndex::UInt64: + return FieldType::OLAP_FIELD_TYPE_UNSIGNED_BIGINT; + case TypeIndex::Int8: + return FieldType::OLAP_FIELD_TYPE_TINYINT; + case TypeIndex::Int16: + return FieldType::OLAP_FIELD_TYPE_SMALLINT; + case TypeIndex::Int32: + return FieldType::OLAP_FIELD_TYPE_INT; + case TypeIndex::Int64: + return FieldType::OLAP_FIELD_TYPE_BIGINT; + case TypeIndex::Float32: + return FieldType::OLAP_FIELD_TYPE_FLOAT; + case TypeIndex::Float64: + return FieldType::OLAP_FIELD_TYPE_DOUBLE; + case TypeIndex::Decimal32: + return FieldType::OLAP_FIELD_TYPE_DECIMAL; + case TypeIndex::Array: + return FieldType::OLAP_FIELD_TYPE_ARRAY; + case TypeIndex::String: + return FieldType::OLAP_FIELD_TYPE_STRING; + case TypeIndex::Date: + return FieldType::OLAP_FIELD_TYPE_DATE; + case TypeIndex::DateTime: + return FieldType::OLAP_FIELD_TYPE_DATETIME; + case TypeIndex::Tuple: + return FieldType::OLAP_FIELD_TYPE_STRUCT; + // TODO add more types + default: + LOG(FATAL) << "unknow type"; + return FieldType::OLAP_FIELD_TYPE_UNKNOWN; + } +} + +Status parse_object_column(ColumnObject& dest, const IColumn& src, bool need_finalize, + const int* row_begin, const int* row_end) { + assert(src.is_column_string()); + const ColumnString* parsing_column {nullptr}; + if (!src.is_nullable()) { + parsing_column = reinterpret_cast(src.get_ptr().get()); + } else { + auto nullable_column = reinterpret_cast(src.get_ptr().get()); + parsing_column = reinterpret_cast( + nullable_column->get_nested_column().get_ptr().get()); + } + std::vector jsons; + if (row_begin != nullptr) { + assert(row_end); + for (auto x = row_begin; x != row_end; ++x) { + StringRef ref = parsing_column->get_data_at(*x); + jsons.push_back(ref); + } + } else { + for (size_t i = 0; i < parsing_column->size(); ++i) { + StringRef ref = parsing_column->get_data_at(i); + jsons.push_back(ref); + } + } + // batch parse + RETURN_IF_ERROR(parse_json_to_variant(dest, jsons)); + + if (need_finalize) { + dest.finalize(); + } + return Status::OK(); +} + +Status parse_object_column(Block& block, size_t position) { + // parse variant column and rewrite column + auto col = block.get_by_position(position).column; + const std::string& col_name = block.get_by_position(position).name; + if (!col->is_column_string()) { + return Status::InvalidArgument("only ColumnString can be parsed to ColumnObject"); + } + vectorized::DataTypePtr type( + std::make_shared("", col->is_nullable())); + auto column_object = type->create_column(); + RETURN_IF_ERROR( + parse_object_column(assert_cast(column_object->assume_mutable_ref()), + *col, true /*need finalize*/, nullptr, nullptr)); + // replace by object + block.safe_get_by_position(position).column = column_object->get_ptr(); + block.safe_get_by_position(position).type = type; + block.safe_get_by_position(position).name = col_name; + return Status::OK(); +} + +void flatten_object(Block& block, size_t pos, bool replace_if_duplicated) { + auto column_object_ptr = + assert_cast(block.get_by_position(pos).column->assume_mutable().get()); + if (column_object_ptr->empty()) { + block.erase(pos); + return; + } + size_t num_rows = column_object_ptr->size(); + assert(block.rows() <= num_rows); + assert(column_object_ptr->is_finalized()); + Columns subcolumns; + DataTypes types; + Names names; + for (auto& subcolumn : column_object_ptr->get_subcolumns()) { + subcolumns.push_back(subcolumn->data.get_finalized_column().get_ptr()); + types.push_back(subcolumn->data.get_least_common_type()); + names.push_back(subcolumn->path.get_path()); + } + block.erase(pos); + for (size_t i = 0; i < subcolumns.size(); ++i) { + // block may already contains this column, eg. key columns, we should ignore + // or replcace the same column from object subcolumn + if (block.has(names[i])) { + if (replace_if_duplicated) { + auto& column_type_name = block.get_by_name(names[i]); + column_type_name.column = subcolumns[i]; + column_type_name.type = types[i]; + } + continue; + } + block.insert(ColumnWithTypeAndName {subcolumns[i], types[i], names[i]}); + } + + // fill default value + for (auto& [column, _1, _2] : block.get_columns_with_type_and_name()) { + if (column->size() < num_rows) { + column->assume_mutable()->insert_many_defaults(num_rows - column->size()); + } + } +} + +Status flatten_object(Block& block, bool replace_if_duplicated) { + auto object_pos = + std::find_if(block.begin(), block.end(), [](const ColumnWithTypeAndName& column) { + return column.type->get_type_id() == TypeIndex::VARIANT; + }); + if (object_pos != block.end()) { + flatten_object(block, object_pos - block.begin(), replace_if_duplicated); + } + return Status::OK(); +} + +bool is_conversion_required_between_integers(const IDataType& lhs, const IDataType& rhs) { + WhichDataType which_lhs(lhs); + WhichDataType which_rhs(rhs); + bool is_native_int = which_lhs.is_native_int() && which_rhs.is_native_int(); + bool is_native_uint = which_lhs.is_native_uint() && which_rhs.is_native_uint(); + return (is_native_int || is_native_uint) && + lhs.get_size_of_value_in_memory() <= rhs.get_size_of_value_in_memory(); +} + +bool is_conversion_required_between_integers(FieldType lhs, FieldType rhs) { + // We only support signed integers for semi-structure data at present + // TODO add unsigned integers + if (lhs == OLAP_FIELD_TYPE_BIGINT) { + return !(rhs == OLAP_FIELD_TYPE_TINYINT || rhs == OLAP_FIELD_TYPE_SMALLINT || + rhs == OLAP_FIELD_TYPE_INT || rhs == OLAP_FIELD_TYPE_BIGINT); + } + if (lhs == OLAP_FIELD_TYPE_INT) { + return !(rhs == OLAP_FIELD_TYPE_TINYINT || rhs == OLAP_FIELD_TYPE_SMALLINT || + rhs == OLAP_FIELD_TYPE_INT); + } + if (lhs == OLAP_FIELD_TYPE_SMALLINT) { + return !(rhs == OLAP_FIELD_TYPE_TINYINT || rhs == OLAP_FIELD_TYPE_SMALLINT); + } + if (lhs == OLAP_FIELD_TYPE_TINYINT) { + return !(rhs == OLAP_FIELD_TYPE_TINYINT); + } + return true; +} + +Status cast_column(const ColumnWithTypeAndName& arg, const DataTypePtr& type, ColumnPtr* result) { + ColumnsWithTypeAndName arguments {arg, + {type->create_column_const_with_default_value(1), type, ""}}; + auto function = SimpleFunctionFactory::instance().get_function("CAST", arguments, type); + Block tmp_block {arguments}; + // the 0 position is input argument, the 1 position is to type argument, the 2 position is result argument + vectorized::ColumnNumbers argnum; + argnum.emplace_back(0); + argnum.emplace_back(1); + size_t result_column = tmp_block.columns(); + tmp_block.insert({nullptr, type, arg.name}); + RETURN_IF_ERROR( + function->execute(nullptr, tmp_block, argnum, result_column, arg.column->size())); + *result = std::move(tmp_block.get_by_position(result_column).column); + return Status::OK(); +} + +static void get_column_def(const vectorized::DataTypePtr& data_type, const std::string& name, + TColumnDef* column) { + if (!name.empty()) { + column->columnDesc.__set_columnName(name); + } + if (data_type->is_nullable()) { + const auto& real_type = static_cast(*data_type); + column->columnDesc.__set_isAllowNull(true); + get_column_def(real_type.get_nested_type(), "", column); + return; + } + column->columnDesc.__set_columnType(to_thrift(get_primitive_type(data_type->get_type_id()))); + if (data_type->get_type_id() == TypeIndex::Array) { + TColumnDef child; + column->columnDesc.__set_children({}); + get_column_def(assert_cast(data_type.get())->get_nested_type(), "", + &child); + column->columnDesc.columnLength = + TabletColumn::get_field_length_by_type(column->columnDesc.columnType, 0); + column->columnDesc.children.push_back(child.columnDesc); + return; + } + if (data_type->get_type_id() == TypeIndex::Tuple) { + // TODO + // auto tuple_type = assert_cast(data_type.get()); + // DCHECK_EQ(tuple_type->get_elements().size(), tuple_type->get_element_names().size()); + // for (size_t i = 0; i < tuple_type->get_elements().size(); ++i) { + // TColumnDef child; + // get_column_def(tuple_type->get_element(i), tuple_type->get_element_names()[i], &child); + // column->columnDesc.children.push_back(child.columnDesc); + // } + // return; + } + if (data_type->get_type_id() == TypeIndex::String) { + return; + } + if (WhichDataType(*data_type).is_simple()) { + column->columnDesc.__set_columnLength(data_type->get_size_of_value_in_memory()); + return; + } +} + +// send an empty add columns rpc, the rpc response will fill with base schema info +// maybe we could seperate this rpc from add columns rpc +Status send_fetch_full_base_schema_view_rpc(FullBaseSchemaView* schema_view) { + TAddColumnsRequest req; + TAddColumnsResult res; + TTabletInfo tablet_info; + req.__set_table_name(schema_view->table_name); + req.__set_db_name(schema_view->db_name); + req.__set_table_id(schema_view->table_id); + auto master_addr = ExecEnv::GetInstance()->master_info()->network_address; + Status rpc_st = ThriftRpcHelper::rpc( + master_addr.hostname, master_addr.port, + [&req, &res](FrontendServiceConnection& client) { client->addColumns(res, req); }, + config::txn_commit_rpc_timeout_ms); + if (!rpc_st.ok()) { + return Status::InternalError("Failed to fetch schema info, encounter rpc failure"); + } + // TODO(lhy) handle more status code + if (res.status.status_code != TStatusCode::OK) { + LOG(WARNING) << "failed to fetch schema info, code:" << res.status.status_code + << ", msg:" << res.status.error_msgs[0]; + return Status::InvalidArgument( + fmt::format("Failed to fetch schema info, {}", res.status.error_msgs[0])); + } + for (const auto& column : res.allColumns) { + schema_view->column_name_to_column[column.column_name] = column; + } + schema_view->schema_version = res.schema_version; + return Status::OK(); +} + +// Do batch add columns schema change +// only the base table supported +Status send_add_columns_rpc(ColumnsWithTypeAndName column_type_names, + FullBaseSchemaView* schema_view) { + if (column_type_names.empty()) { + return Status::OK(); + } + TAddColumnsRequest req; + TAddColumnsResult res; + TTabletInfo tablet_info; + req.__set_table_name(schema_view->table_name); + req.__set_db_name(schema_view->db_name); + req.__set_table_id(schema_view->table_id); + // TODO(lhy) more configurable + req.__set_allow_type_conflict(true); + for (const auto& column_type_name : column_type_names) { + TColumnDef col; + get_column_def(column_type_name.type, column_type_name.name, &col); + req.addColumns.push_back(col); + } + auto master_addr = ExecEnv::GetInstance()->master_info()->network_address; + Status rpc_st = ThriftRpcHelper::rpc( + master_addr.hostname, master_addr.port, + [&req, &res](FrontendServiceConnection& client) { client->addColumns(res, req); }, + config::txn_commit_rpc_timeout_ms); + if (!rpc_st.ok()) { + return Status::InternalError("Failed to do schema change, rpc error"); + } + // TODO(lhy) handle more status code + if (res.status.status_code != TStatusCode::OK) { + LOG(WARNING) << "failed to do schema change, code:" << res.status.status_code + << ", msg:" << res.status.error_msgs[0]; + return Status::InvalidArgument( + fmt::format("Failed to do schema change, {}", res.status.error_msgs[0])); + } + size_t sz = res.allColumns.size(); + if (sz < column_type_names.size()) { + return Status::InternalError( + fmt::format("Unexpected result columns {}, expected at least {}", + res.allColumns.size(), column_type_names.size())); + } + for (const auto& column : res.allColumns) { + schema_view->column_name_to_column[column.column_name] = column; + } + schema_view->schema_version = res.schema_version; + return Status::OK(); +} + +template +void align_block_by_name_and_type(MutableBlock* mblock, const Block* block, size_t row_cnt, + ColumnInserterFn inserter) { + assert(!mblock->get_names().empty()); + const auto& names = mblock->get_names(); + [[maybe_unused]] const auto& data_types = mblock->data_types(); + size_t num_rows = mblock->rows(); + for (size_t i = 0; i < mblock->columns(); ++i) { + auto& dst = mblock->get_column_by_position(i); + if (!block->has(names[i])) { + dst->insert_many_defaults(row_cnt); + } else { + assert(data_types[i]->equals(*block->get_by_name(names[i]).type)); + const auto& src = *(block->get_by_name(names[i]).column.get()); + inserter(src, dst); + } + } + for (const auto& [column, type, name] : *block) { + // encounter a new column + if (!mblock->has(name)) { + auto new_column = type->create_column(); + new_column->insert_many_defaults(num_rows); + inserter(*column.get(), new_column); + mblock->mutable_columns().push_back(std::move(new_column)); + mblock->data_types().push_back(type); + mblock->get_names().push_back(name); + } + } + +#ifndef NDEBUG + // Check all columns rows matched + num_rows = mblock->rows(); + for (size_t i = 0; i < mblock->columns(); ++i) { + DCHECK_EQ(mblock->mutable_columns()[i]->size(), num_rows); + } +#endif +} + +void align_block_by_name_and_type(MutableBlock* mblock, const Block* block, const int* row_begin, + const int* row_end) { + align_block_by_name_and_type(mblock, block, row_end - row_begin, + [row_begin, row_end](const IColumn& src, MutableColumnPtr& dst) { + dst->insert_indices_from(src, row_begin, row_end); + }); +} +void align_block_by_name_and_type(MutableBlock* mblock, const Block* block, size_t row_begin, + size_t length) { + align_block_by_name_and_type(mblock, block, length, + [row_begin, length](const IColumn& src, MutableColumnPtr& dst) { + dst->insert_range_from(src, row_begin, length); + }); +} + +void align_append_block_by_selector(MutableBlock* mblock, const Block* block, + const IColumn::Selector& selector) { + // append by selector with alignment + assert(!mblock->get_names().empty()); + align_block_by_name_and_type(mblock, block, selector.size(), + [&selector](const IColumn& src, MutableColumnPtr& dst) { + src.append_data_by_selector(dst, selector); + }); +} + +void LocalSchemaChangeRecorder::add_extended_columns(const TabletColumn& new_column, + int32_t schema_version) { + std::lock_guard lock(_lock); + _schema_version = std::max(_schema_version, schema_version); + auto it = _extended_columns.find(new_column.name()); + if (it != _extended_columns.end()) { + return; + } + _extended_columns.emplace_hint(it, new_column.name(), new_column); +} + +bool LocalSchemaChangeRecorder::has_extended_columns() { + std::lock_guard lock(_lock); + return !_extended_columns.empty(); +} + +std::map LocalSchemaChangeRecorder::copy_extended_columns() { + std::lock_guard lock(_lock); + return _extended_columns; +} + +const TabletColumn& LocalSchemaChangeRecorder::column(const std::string& col_name) { + std::lock_guard lock(_lock); + assert(_extended_columns.find(col_name) != _extended_columns.end()); + return _extended_columns[col_name]; +} + +int32_t LocalSchemaChangeRecorder::schema_version() { + std::lock_guard lock(_lock); + return _schema_version; +} + +} // namespace doris::vectorized::schema_util diff --git a/be/src/vec/common/schema_util.h b/be/src/vec/common/schema_util.h new file mode 100644 index 00000000000000..3531dc129ba918 --- /dev/null +++ b/be/src/vec/common/schema_util.h @@ -0,0 +1,128 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include +#include +#include +#include +#include + +#include "olap/tablet_schema.h" + +namespace doris { +class LocalSchemaChangeRecorder; +} + +namespace doris::vectorized::schema_util { +/// Returns number of dimensions in Array type. 0 if type is not array. +size_t get_number_of_dimensions(const IDataType& type); + +/// Returns number of dimensions in Array column. 0 if column is not array. +size_t get_number_of_dimensions(const IColumn& column); + +/// Returns type of scalars of Array of arbitrary dimensions. +DataTypePtr get_base_type_of_array(const DataTypePtr& type); + +/// Returns Array with requested number of dimensions and no scalars. +Array create_empty_array_field(size_t num_dimensions); + +// NOTICE: the last column must be dynamic column +// 1. The dynamic column will be parsed to ColumnObject and the parsed column will +// be flattened to multiple subcolumns, thus the dynamic schema is infered from the +// dynamic column. +// 2. Schema change which is add columns will be performed if the infered schema is +// different from the original tablet schema, new columns added to schema change history +Status parse_and_expand_dynamic_column(Block& block, const TabletSchema& schema_hints, + LocalSchemaChangeRecorder* history); + +Status parse_object_column(Block& block, size_t position); + +Status parse_object_column(ColumnObject& dest, const IColumn& src, bool need_finalize, + const int* row_begin, const int* row_end); + +// Object column will be flattened and if replace_if_duplicated +// the original column in the block will be replaced with the subcolumn +// from object column.Also if column in block is empty, it will be filled +// with num_rows of default values +Status flatten_object(Block& block, bool replace_if_duplicated); + +/// If both of types are signed/unsigned integers and size of left field type +/// is less than right type, we don't need to convert field, +/// because all integer fields are stored in Int64/UInt64. +bool is_conversion_required_between_integers(const IDataType& lhs, const IDataType& rhs); +bool is_conversion_required_between_integers(FieldType lhs, FieldType rhs); + +// Cast column to type +Status cast_column(const ColumnWithTypeAndName& arg, const DataTypePtr& type, ColumnPtr* result); + +// Align block schema with tablet schema +// eg. +// Block: col1(int), col2(string) +// Schema: col1(double), col3(date) +// 1. col1(int) in block which type missmatch with schema col1 will be converted to double +// 2. col2 in block which missing in current schema will launch a schema change rpc +// 3. col3 in schema which missing in block will be ignored +// After schema changed, schame change history will add new columns +Status align_block_with_schema(const TabletSchema& schema, int64_t table_id /*for schema change*/, + Block& block, LocalSchemaChangeRecorder* history); +// record base schema column infos +// maybe use col_unique_id as key in the future +// but for dynamic table, column name if ok +struct FullBaseSchemaView { + phmap::flat_hash_map column_name_to_column; + int32_t schema_version = -1; + int32_t table_id = 0; + std::string table_name; + std::string db_name; + + bool empty() { return column_name_to_column.empty() && schema_version == -1; } +}; + +Status send_add_columns_rpc(ColumnsWithTypeAndName column_type_names, + FullBaseSchemaView* schema_view); + +Status send_fetch_full_base_schema_view_rpc(FullBaseSchemaView* schema_view); + +// block alignment +// TODO using column_unique_id instead of names +void align_block_by_name_and_type(MutableBlock* mblock, const Block* block, const int* row_begin, + const int* row_end); +void align_block_by_name_and_type(MutableBlock* mblock, const Block* block, size_t row_begin, + size_t length); + +void align_append_block_by_selector(MutableBlock* mblock, const Block* block, + const IColumn::Selector& selector); + +// For tracking local schema change during load procedure +class LocalSchemaChangeRecorder { +public: + void add_extended_columns(const TabletColumn& new_column, int32_t schema_version); + bool has_extended_columns(); + std::map copy_extended_columns(); + const TabletColumn& column(const std::string& col_name); + int32_t schema_version(); + +private: + std::mutex _lock; + int32_t _schema_version = -1; + std::map _extended_columns; +}; + +} // namespace doris::vectorized::schema_util diff --git a/be/src/vec/common/sip_hash.h b/be/src/vec/common/sip_hash.h index 21f1870e83abc7..e9afdca10a6a9d 100644 --- a/be/src/vec/common/sip_hash.h +++ b/be/src/vec/common/sip_hash.h @@ -198,6 +198,12 @@ class SipHash { finalize(); return v0 ^ v1 ^ v2 ^ v3; } + + template + ALWAYS_INLINE void get128(T& dst) { + static_assert(sizeof(T) == 16); + get128(reinterpret_cast(&dst)); + } }; #undef ROTL diff --git a/be/src/vec/common/string_ref.h b/be/src/vec/common/string_ref.h index 1e98addf9ee27b..9447ccf26fd9c6 100644 --- a/be/src/vec/common/string_ref.h +++ b/be/src/vec/common/string_ref.h @@ -242,6 +242,8 @@ struct StringRef { // Trims leading and trailing spaces. StringRef trim() const; + bool empty() const { return size == 0; } + // support for type_limit static constexpr char MIN_CHAR = 0; static constexpr char MAX_CHAR = char( diff --git a/be/src/vec/common/typeid_cast.h b/be/src/vec/common/typeid_cast.h index 60ef9743d9fdfe..0284c911068edb 100644 --- a/be/src/vec/common/typeid_cast.h +++ b/be/src/vec/common/typeid_cast.h @@ -29,6 +29,14 @@ #include "vec/common/demangle.h" #include "vec/common/exception.h" +#define TYPEID_MAP(_A) \ + template <> \ + inline constexpr TypeIndex TypeToTypeIndex<_A> = TypeIndex::_A; \ + template <> \ + struct TypeIndexToTypeHelper : std::true_type { \ + using T = _A; \ + }; + /** Checks type by comparing typeid. * The exact match of the type is checked. That is, cast to the ancestor will be unsuccessful. * In the rest, behaves like a dynamic_cast. diff --git a/be/src/vec/core/accurate_comparison.h b/be/src/vec/core/accurate_comparison.h index e52cc4ef6596c5..10e961311ca9c3 100644 --- a/be/src/vec/core/accurate_comparison.h +++ b/be/src/vec/core/accurate_comparison.h @@ -179,8 +179,9 @@ constexpr doris::vectorized::Int64 MAX_INT64_WITH_EXACT_FLOAT64_REPR = 900719925 template <> inline bool greaterOp( doris::vectorized::Float64 f, doris::vectorized::Int64 i) { - if (-MAX_INT64_WITH_EXACT_FLOAT64_REPR <= i && i <= MAX_INT64_WITH_EXACT_FLOAT64_REPR) + if (-MAX_INT64_WITH_EXACT_FLOAT64_REPR <= i && i <= MAX_INT64_WITH_EXACT_FLOAT64_REPR) { return f > static_cast(i); + } return (f >= static_cast( std::numeric_limits< @@ -193,8 +194,9 @@ inline bool greaterOp( template <> inline bool greaterOp( doris::vectorized::Int64 i, doris::vectorized::Float64 f) { - if (-MAX_INT64_WITH_EXACT_FLOAT64_REPR <= i && i <= MAX_INT64_WITH_EXACT_FLOAT64_REPR) + if (-MAX_INT64_WITH_EXACT_FLOAT64_REPR <= i && i <= MAX_INT64_WITH_EXACT_FLOAT64_REPR) { return f < static_cast(i); + } return (f < static_cast( std::numeric_limits::min())) || @@ -206,8 +208,9 @@ inline bool greaterOp( template <> inline bool greaterOp( doris::vectorized::Float64 f, doris::vectorized::UInt64 u) { - if (u <= static_cast(MAX_INT64_WITH_EXACT_FLOAT64_REPR)) + if (u <= static_cast(MAX_INT64_WITH_EXACT_FLOAT64_REPR)) { return f > static_cast(u); + } return (f >= static_cast( std::numeric_limits::max())) || @@ -217,8 +220,9 @@ inline bool greaterOp( template <> inline bool greaterOp( doris::vectorized::UInt64 u, doris::vectorized::Float64 f) { - if (u <= static_cast(MAX_INT64_WITH_EXACT_FLOAT64_REPR)) + if (u <= static_cast(MAX_INT64_WITH_EXACT_FLOAT64_REPR)) { return static_cast(u) > f; + } return (f < 0) || (f < static_cast( std::numeric_limits::max()) && @@ -370,8 +374,9 @@ inline bool greaterOp(doris::vectorized::Int128 i, doris::vectorized::Float64 f) static constexpr __int128 max_int128 = (__int128(0x7fffffffffffffffll) << 64) + 0xffffffffffffffffll; - if (-MAX_INT64_WITH_EXACT_FLOAT64_REPR <= i && i <= MAX_INT64_WITH_EXACT_FLOAT64_REPR) + if (-MAX_INT64_WITH_EXACT_FLOAT64_REPR <= i && i <= MAX_INT64_WITH_EXACT_FLOAT64_REPR) { return static_cast(i) > f; + } return (f < static_cast(min_int128)) || (f < static_cast(max_int128) && @@ -383,8 +388,9 @@ inline bool greaterOp(doris::vectorized::Float64 f, doris::vectorized::Int128 i) static constexpr __int128 max_int128 = (__int128(0x7fffffffffffffffll) << 64) + 0xffffffffffffffffll; - if (-MAX_INT64_WITH_EXACT_FLOAT64_REPR <= i && i <= MAX_INT64_WITH_EXACT_FLOAT64_REPR) + if (-MAX_INT64_WITH_EXACT_FLOAT64_REPR <= i && i <= MAX_INT64_WITH_EXACT_FLOAT64_REPR) { return f > static_cast(i); + } return (f >= static_cast(max_int128)) || (f > static_cast(min_int128) && @@ -435,7 +441,9 @@ inline bool_if_safe_conversion lessOp(A a, B b) { template inline bool_if_not_safe_conversion lessOrEqualsOp(A a, B b) { - if (is_nan(a) || is_nan(b)) return false; + if (is_nan(a) || is_nan(b)) { + return false; + } return !greaterOp(a, b); } @@ -446,7 +454,9 @@ inline bool_if_safe_conversion lessOrEqualsOp(A a, B b) { template inline bool_if_not_safe_conversion greaterOrEqualsOp(A a, B b) { - if (is_nan(a) || is_nan(b)) return false; + if (is_nan(a) || is_nan(b)) { + return false; + } return !greaterOp(b, a); } @@ -456,22 +466,38 @@ inline bool_if_safe_conversion greaterOrEqualsOp(A a, B b) { } /// Converts numeric to an equal numeric of other type. -template +/// When `strict` is `true` check that result exactly same as input, otherwise just check overflow +template inline bool convertNumeric(From value, To& result) { /// If the type is actually the same it's not necessary to do any checks. if constexpr (std::is_same_v) { result = value; return true; } - - /// Note that NaNs doesn't compare equal to anything, but they are still in range of any Float type. - if (is_nan(value) && std::is_floating_point_v) { - result = value; - return true; + if constexpr (std::is_floating_point_v && std::is_floating_point_v) { + /// Note that NaNs doesn't compare equal to anything, but they are still in range of any Float type. + if (is_nan(value)) { + result = value; + return true; + } + if (value == std::numeric_limits::infinity()) { + result = std::numeric_limits::infinity(); + return true; + } + if (value == -std::numeric_limits::infinity()) { + result = -std::numeric_limits::infinity(); + return true; + } + } + if (greaterOp(value, std::numeric_limits::max()) || + lessOp(value, std::numeric_limits::lowest())) { + return false; } - result = static_cast(value); - return equalsOp(value, result); + if constexpr (strict) { + return equalsOp(value, result); + } + return true; } } // namespace accurate diff --git a/be/src/vec/core/block.cpp b/be/src/vec/core/block.cpp index b608a6ac4928a2..ce54efd5a4b348 100644 --- a/be/src/vec/core/block.cpp +++ b/be/src/vec/core/block.cpp @@ -38,6 +38,8 @@ #include "vec/columns/column_vector.h" #include "vec/columns/columns_number.h" #include "vec/common/assert_cast.h" +#include "vec/common/exception.h" +#include "vec/common/schema_util.h" #include "vec/common/string_ref.h" #include "vec/common/typeid_cast.h" #include "vec/data_types/data_type_factory.hpp" @@ -476,6 +478,7 @@ Block Block::clone_empty() const { for (const auto& elem : data) { res.insert(elem.clone_empty()); } + res.set_block_type(_type); return res; } @@ -530,6 +533,7 @@ Block Block::clone_with_columns(MutableColumns&& columns) const { for (size_t i = 0; i < num_columns; ++i) { res.insert({std::move(columns[i]), data[i].type, data[i].name}); } + res.set_block_type(_type); return res; } @@ -549,7 +553,7 @@ Block Block::clone_with_columns(const Columns& columns) const { for (size_t i = 0; i < num_columns; ++i) { res.insert({columns[i], data[i].type, data[i].name}); } - + res.set_block_type(_type); return res; } @@ -560,7 +564,7 @@ Block Block::clone_without_columns() const { for (size_t i = 0; i < num_columns; ++i) { res.insert({nullptr, data[i].type, data[i].name}); } - + res.set_block_type(_type); return res; } @@ -678,11 +682,14 @@ Block Block::copy_block(const std::vector& column_offset) const { return columns_with_type_and_name; } -void Block::append_block_by_selector(MutableColumns& columns, - const IColumn::Selector& selector) const { - DCHECK(data.size() == columns.size()); +void Block::append_block_by_selector(MutableBlock* dst, const IColumn::Selector& selector) const { + if (UNLIKELY(dst->get_block_type() == BlockType::DYNAMIC)) { + schema_util::align_append_block_by_selector(dst, this, selector); + return; + } + DCHECK(data.size() == dst->mutable_columns().size()); for (size_t i = 0; i < data.size(); i++) { - data[i].column->append_data_by_selector(columns[i], selector); + data[i].column->append_data_by_selector(dst->mutable_columns()[i], selector); } } @@ -869,6 +876,10 @@ void MutableBlock::add_row(const Block* block, int row) { } void MutableBlock::add_rows(const Block* block, const int* row_begin, const int* row_end) { + if (UNLIKELY(_type == BlockType::DYNAMIC)) { + schema_util::align_block_by_name_and_type(this, block, row_begin, row_end); + return; + } auto& block_data = block->get_columns_with_type_and_name(); for (size_t i = 0; i < _columns.size(); ++i) { auto& dst = _columns[i]; @@ -878,6 +889,10 @@ void MutableBlock::add_rows(const Block* block, const int* row_begin, const int* } void MutableBlock::add_rows(const Block* block, size_t row_begin, size_t length) { + if (UNLIKELY(_type == BlockType::DYNAMIC)) { + schema_util::align_block_by_name_and_type(this, block, row_begin, length); + return; + } auto& block_data = block->get_columns_with_type_and_name(); for (size_t i = 0; i < _columns.size(); ++i) { auto& dst = _columns[i]; diff --git a/be/src/vec/core/block.h b/be/src/vec/core/block.h index 1e32b506b49a98..bd07512a6bf6c0 100644 --- a/be/src/vec/core/block.h +++ b/be/src/vec/core/block.h @@ -46,13 +46,26 @@ struct TypeDescriptor; namespace vectorized { +// DYNAMIC block is a special type of Block. +// It could extends it's structure by align with +// other blocks by add_rows, merge, append_block_by_selector ... +// eg. +// BlockA: A(int) | B(double) | C(float) | D(string) +// BlockB: E(date) | F(int) +// when BlockA.add_rows/merge/... with BlockB +// then BlockA structure will become: +// A(int) | B(double) | C(float) | D(string) | E(date) | F(int) +// Both E & F are added to BlockA with same rows, and missing columns are +// filled with default values +enum class BlockType { NORMAL, DYNAMIC }; + /** Container for set of columns for bunch of rows in memory. * This is unit of data processing. * Also contains metadata - data types of columns and their names * (either original names from a table, or generated names during temporary calculations). * Allows to insert, remove columns in arbitrary position, to change order of columns. */ - +class MutableBlock; class Block { private: using Container = ColumnsWithTypeAndName; @@ -66,6 +79,7 @@ class Block { int64_t _decompressed_bytes = 0; mutable int64_t _compress_time_ns = 0; + BlockType _type {BlockType::NORMAL}; public: Block() = default; @@ -147,6 +161,9 @@ class Block { element.column = element.column->convert_to_full_column_if_const(); } + void set_block_type(BlockType type) { _type = type; } + BlockType get_block_type() { return _type; } + ColumnWithTypeAndName& safe_get_by_position(size_t position); const ColumnWithTypeAndName& safe_get_by_position(size_t position) const; @@ -257,7 +274,7 @@ class Block { // copy a new block by the offset column Block copy_block(const std::vector& column_offset) const; - void append_block_by_selector(MutableColumns& columns, const IColumn::Selector& selector) const; + void append_block_by_selector(MutableBlock* dst, const IColumn::Selector& selector) const; static void filter_block_internal(Block* block, const std::vector& columns_to_filter, const IColumn::Filter& filter); @@ -386,8 +403,12 @@ class MutableBlock { using IndexByName = phmap::flat_hash_map; IndexByName index_by_name; + BlockType _type {BlockType::NORMAL}; public: + void set_block_type(BlockType type) { _type = type; } + BlockType get_block_type() { return _type; } + static MutableBlock build_mutable_block(Block* block) { return block == nullptr ? MutableBlock() : MutableBlock(block); } @@ -400,13 +421,15 @@ class MutableBlock { MutableBlock(Block* block) : _columns(block->mutate_columns()), _data_types(block->get_data_types()), - _names(block->get_names()) { + _names(block->get_names()), + _type(block->get_block_type()) { initialize_index_by_name(); } MutableBlock(Block&& block) : _columns(block.mutate_columns()), _data_types(block.get_data_types()), - _names(block.get_names()) { + _names(block.get_names()), + _type(block.get_block_type()) { initialize_index_by_name(); } @@ -414,6 +437,7 @@ class MutableBlock { _columns = std::move(m_block._columns); _data_types = std::move(m_block._data_types); _names = std::move(m_block._names); + _type = m_block.get_block_type(); initialize_index_by_name(); } @@ -476,6 +500,8 @@ class MutableBlock { template void merge(T&& block) { + // merge is not supported in dynamic block + DCHECK(_type != BlockType::DYNAMIC); if (_columns.size() == 0 && _data_types.size() == 0) { _data_types = block.get_data_types(); _names = block.get_names(); diff --git a/be/src/vec/core/field.h b/be/src/vec/core/field.h index 78ac881a19a062..cba06ab4e32745 100644 --- a/be/src/vec/core/field.h +++ b/be/src/vec/core/field.h @@ -86,9 +86,16 @@ using FieldVector = std::vector; DEFINE_FIELD_VECTOR(Array); DEFINE_FIELD_VECTOR(Tuple); DEFINE_FIELD_VECTOR(Map); - #undef DEFINE_FIELD_VECTOR +using FieldMap = std::map>; +#define DEFINE_FIELD_MAP(X) \ + struct X : public FieldMap { \ + using FieldMap::FieldMap; \ + } +DEFINE_FIELD_MAP(VariantMap); +#undef DEFINE_FIELD_MAP + struct AggregateFunctionStateData { String name; /// Name with arguments. String data; @@ -310,6 +317,7 @@ class Field { JSONB = 23, Decimal128I = 24, Map = 25, + VariantMap = 26, }; static const int MIN_NON_POD = 16; @@ -350,6 +358,8 @@ class Field { return "AggregateFunctionState"; case FixedLengthObject: return "FixedLengthObject"; + case VariantMap: + return "VariantMap"; } LOG(FATAL) << "Bad type of Field"; @@ -526,6 +536,8 @@ class Field { return get() < rhs.get(); case Types::FixedLengthObject: break; + case Types::VariantMap: + return get() < rhs.get(); } LOG(FATAL) << "Bad type of Field"; @@ -573,6 +585,8 @@ class Field { return get() <= rhs.get(); case Types::FixedLengthObject: break; + case Types::VariantMap: + return get() <= rhs.get(); } LOG(FATAL) << "Bad type of Field"; return {}; @@ -616,6 +630,8 @@ class Field { return get() == rhs.get(); case Types::FixedLengthObject: break; + case Types::VariantMap: + return get() == rhs.get(); } CHECK(false) << "Bad type of Field"; @@ -711,6 +727,9 @@ class Field { case Types::FixedLengthObject: LOG(FATAL) << "FixedLengthObject not supported"; break; + case Types::VariantMap: + f(field.template get()); + return; } } @@ -771,6 +790,9 @@ class Field { case Types::AggregateFunctionState: destroy(); break; + case Types::VariantMap: + destroy(); + break; default: break; } @@ -788,33 +810,57 @@ class Field { #undef DBMS_MIN_FIELD_SIZE +template <> +struct TypeId { + static constexpr const TypeIndex value = TypeIndex::AggregateFunction; +}; +template <> +struct TypeId { + static constexpr const TypeIndex value = TypeIndex::Tuple; +}; +template <> +struct TypeId> { + static constexpr const TypeIndex value = TypeIndex::Decimal32; +}; +template <> +struct TypeId> { + static constexpr const TypeIndex value = TypeIndex::Decimal64; +}; +template <> +struct TypeId> { + static constexpr const TypeIndex value = TypeIndex::Decimal128; +}; +template <> +struct TypeId> { + static constexpr const TypeIndex value = TypeIndex::Decimal128I; +}; template <> struct Field::TypeToEnum { - static const Types::Which value = Types::Null; + static constexpr Types::Which value = Types::Null; }; template <> struct Field::TypeToEnum { - static const Types::Which value = Types::UInt64; + static constexpr Types::Which value = Types::UInt64; }; template <> struct Field::TypeToEnum { - static const Types::Which value = Types::UInt128; + static constexpr Types::Which value = Types::UInt128; }; template <> struct Field::TypeToEnum { - static const Types::Which value = Types::Int64; + static constexpr Types::Which value = Types::Int64; }; template <> struct Field::TypeToEnum { - static const Types::Which value = Types::Int128; + static constexpr Types::Which value = Types::Int128; }; template <> struct Field::TypeToEnum { - static const Types::Which value = Types::Float64; + static constexpr Types::Which value = Types::Float64; }; template <> struct Field::TypeToEnum { - static const Types::Which value = Types::String; + static constexpr Types::Which value = Types::String; }; template <> struct Field::TypeToEnum { @@ -822,11 +868,11 @@ struct Field::TypeToEnum { }; template <> struct Field::TypeToEnum { - static const Types::Which value = Types::Array; + static constexpr Types::Which value = Types::Array; }; template <> struct Field::TypeToEnum { - static const Types::Which value = Types::Tuple; + static constexpr Types::Which value = Types::Tuple; }; template <> struct Field::TypeToEnum { @@ -834,23 +880,27 @@ struct Field::TypeToEnum { }; template <> struct Field::TypeToEnum> { - static const Types::Which value = Types::Decimal32; + static constexpr Types::Which value = Types::Decimal32; }; template <> struct Field::TypeToEnum> { - static const Types::Which value = Types::Decimal64; + static constexpr Types::Which value = Types::Decimal64; }; template <> struct Field::TypeToEnum> { - static const Types::Which value = Types::Decimal128; + static constexpr Types::Which value = Types::Decimal128; }; template <> struct Field::TypeToEnum> { - static const Types::Which value = Types::Decimal128I; + static constexpr Types::Which value = Types::Decimal128I; }; template <> struct Field::TypeToEnum { - static const Types::Which value = Types::AggregateFunctionState; + static constexpr Types::Which value = Types::AggregateFunctionState; +}; +template <> +struct Field::TypeToEnum { + static constexpr Types::Which value = Types::VariantMap; }; template <> @@ -917,6 +967,10 @@ template <> struct Field::EnumToType { using Type = DecimalField; }; +template <> +struct Field::EnumToType { + using Type = VariantMap; +}; template T get(const Field& field) { @@ -946,6 +1000,11 @@ template <> struct TypeName { static std::string get() { return "Tuple"; } }; + +template <> +struct TypeName { + static std::string get() { return "VariantMap"; } +}; template <> struct TypeName { static std::string get() { return "Map"; } @@ -994,6 +1053,11 @@ struct NearestFieldTypeImpl { using Type = Int64; }; +template <> +struct NearestFieldTypeImpl { + using Type = VariantMap; +}; + /// long and long long are always different types that may behave identically or not. /// This is different on Linux and Mac. template <> @@ -1090,6 +1154,11 @@ struct NearestFieldTypeImpl { using Type = Null; }; +template <> +struct NearestFieldTypeImpl { + using Type = String; +}; + template <> struct NearestFieldTypeImpl { using Type = AggregateFunctionStateData; diff --git a/be/src/vec/core/types.h b/be/src/vec/core/types.h index 8e62fed5973955..f8f02032087f3e 100644 --- a/be/src/vec/core/types.h +++ b/be/src/vec/core/types.h @@ -84,6 +84,7 @@ enum class TypeIndex { Decimal128I, Map, Struct, + VARIANT, }; struct Consted { @@ -247,6 +248,10 @@ template <> struct TypeId { static constexpr const TypeIndex value = TypeIndex::Float64; }; +template <> +struct TypeId { + static constexpr const TypeIndex value = TypeIndex::String; +}; /// Not a data type in database, defined just for convenience. using Strings = std::vector; @@ -587,6 +592,8 @@ inline const char* getTypeName(TypeIndex idx) { return "AggregateFunction"; case TypeIndex::LowCardinality: return "LowCardinality"; + case TypeIndex::VARIANT: + return "Variant"; case TypeIndex::BitMap: return TypeName::get(); case TypeIndex::HLL: diff --git a/be/src/vec/data_types/convert_field_to_type.cpp b/be/src/vec/data_types/convert_field_to_type.cpp new file mode 100644 index 00000000000000..c516702678787a --- /dev/null +++ b/be/src/vec/data_types/convert_field_to_type.cpp @@ -0,0 +1,248 @@ +// 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. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/Interpreters/convert_field_to_type.cpp +// and modified by Doris + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +// #include +namespace doris::vectorized { +/** Checking for a `Field from` of `From` type falls to a range of values of type `To`. + * `From` and `To` - numeric types. They can be floating-point types. + * `From` is one of UInt64, Int64, Float64, + * whereas `To` can also be 8, 16, 32 bit. + * + * If falls into a range, then `from` is converted to the `Field` closest to the `To` type. + * If not, return Field(Null). + */ + +/** simple types of implementation of visitor to string*/ +// TODO support more types +class FieldVisitorToStringSimple : public StaticVisitor { +public: + String operator()(const Null& x) const { return "NULL"; } + String operator()(const UInt64& x) const { return std::to_string(x); } + String operator()(const Int64& x) const { return std::to_string(x); } + 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"; } + [[noreturn]] String operator()(const Array& x) const { LOG(FATAL) << "not implemeted"; } + [[noreturn]] String operator()(const Tuple& x) const { LOG(FATAL) << "not implemeted"; } + [[noreturn]] String operator()(const DecimalField& x) const { + LOG(FATAL) << "not implemeted"; + } + [[noreturn]] String operator()(const DecimalField& x) const { + LOG(FATAL) << "not implemeted"; + } + [[noreturn]] String operator()(const DecimalField& x) const { + LOG(FATAL) << "not implemeted"; + } + [[noreturn]] String operator()(const DecimalField& x) const { + LOG(FATAL) << "not implemeted"; + } + [[noreturn]] String operator()(const AggregateFunctionStateData& x) const { + LOG(FATAL) << "not implemeted"; + } +}; + +namespace { +template +Field convert_numeric_type_impl(const Field& from) { + To result; + if (!accurate::convertNumeric(from.get(), result)) { + return {}; + } + return result; +} +template +Status convert_numric_type(const Field& from, const IDataType& type, Field* to) { + if (from.get_type() == Field::Types::UInt64) { + *to = convert_numeric_type_impl(from); + } else if (from.get_type() == Field::Types::Int64) { + *to = convert_numeric_type_impl(from); + } else if (from.get_type() == Field::Types::Float64) { + *to = convert_numeric_type_impl(from); + } else if (from.get_type() == Field::Types::UInt128) { + // *to = convert_numeric_type_impl(from); + } else if (from.get_type() == Field::Types::Int128) { + *to = convert_numeric_type_impl(from); + } else { + return Status::InvalidArgument( + fmt::format("Type mismatch in IN or VALUES section. Expected: {}. Got: {}", + type.get_name(), from.get_type())); + } + return Status::OK(); +} + +Status convert_field_to_typeImpl(const Field& src, const IDataType& type, + const IDataType* from_type_hint, Field* to) { + if (from_type_hint && from_type_hint->equals(type)) { + *to = src; + return Status::OK(); + } + WhichDataType which_type(type); + // TODO add more types + if (type.is_value_represented_by_number() && src.get_type() != Field::Types::String) { + if (which_type.is_uint8()) { + return convert_numric_type(src, type, to); + } + if (which_type.is_uint16()) { + return convert_numric_type(src, type, to); + } + if (which_type.is_uint32()) { + return convert_numric_type(src, type, to); + } + if (which_type.is_uint64()) { + return convert_numric_type(src, type, to); + } + if (which_type.is_uint128()) { + // return convert_numric_type(src, type, to); + } + if (which_type.is_int8()) { + return convert_numric_type(src, type, to); + } + if (which_type.is_int16()) { + return convert_numric_type(src, type, to); + } + if (which_type.is_int32()) { + return convert_numric_type(src, type, to); + } + if (which_type.is_int64()) { + return convert_numric_type(src, type, to); + } + if (which_type.is_int128()) { + return convert_numric_type(src, type, to); + } + if (which_type.is_float32()) { + return convert_numric_type(src, type, to); + } + if (which_type.is_float64()) { + return convert_numric_type(src, type, to); + } + if ((which_type.is_date() || which_type.is_date_time()) && + src.get_type() == Field::Types::UInt64) { + /// We don't need any conversion UInt64 is under type of Date and DateTime + *to = src; + return Status::OK(); + } + } else if (which_type.is_string_or_fixed_string()) { + if (src.get_type() == Field::Types::String) { + *to = src; + return Status::OK(); + } + // TODO this is a very simple translator, support more complex types + *to = apply_visitor(FieldVisitorToStringSimple(), src); + return Status::OK(); + } else if (const DataTypeArray* type_array = typeid_cast(&type)) { + if (src.get_type() == Field::Types::Array) { + const Array& src_arr = src.get(); + size_t src_arr_size = src_arr.size(); + const auto& element_type = *(type_array->get_nested_type()); + Array res(src_arr_size); + for (size_t i = 0; i < src_arr_size; ++i) { + RETURN_IF_ERROR(convert_field_to_type(src_arr[i], element_type, &res[i])); + if (res[i].is_null() && !element_type.is_nullable()) { + return Status::InvalidArgument( + fmt::format("Cannot convert NULL to {}", element_type.get_name())); + } + } + *to = Field(res); + return Status::OK(); + } + } + // else if (const DataTypeTuple* type_tuple = typeid_cast(&type)) { + // if (src.get_type() == Field::Types::Tuple) { + // const auto& src_tuple = src.get(); + // size_t src_tuple_size = src_tuple.size(); + // size_t dst_tuple_size = type_tuple->get_elements().size(); + // if (dst_tuple_size != src_tuple_size) { + // return Status::InvalidArgument("Bad size of tuple in IN or VALUES section"); + // } + // Tuple res(dst_tuple_size); + // bool have_unconvertible_element = false; + // for (size_t i = 0; i < dst_tuple_size; ++i) { + // const auto& element_type = *(type_tuple->get_elements()[i]); + // RETURN_IF_ERROR(convert_field_to_type(src_tuple[i], element_type, &res[i])); + // if (!res[i].is_null() || element_type.is_nullable()) continue; + // /* + // * Either the source element was Null, or the conversion did not + // * succeed, because the source and the requested types of the + // * element are compatible, but the value is not convertible + // * (e.g. trying to convert -1 from Int8 to UInt8). In these + // * cases, consider the whole tuple also compatible but not + // * convertible. According to the specification of this function, + // * we must return Null in this case. + // * + // * The following elements might be not even compatible, so it + // * makes sense to check them to detect user errors. Remember + // * that there is an unconvertible element, and try to process + // * the remaining ones. The convert_field_to_type for each element + // * will throw if it detects incompatibility. + // */ + // have_unconvertible_element = true; + // } + // if (have_unconvertible_element) { + // return Status::InvalidArgument(fmt::format("Cannot convert {} to {}", + // src.get_type_name(), type.get_name())); + // } + // *to = Field(res); + // return Status::OK(); + // } + // } + return Status::InvalidArgument( + fmt::format("Type mismatch in IN or VALUES section. Expected: {}. Got: {}", + type.get_name(), src.get_type())); +} +} // namespace +Status convert_field_to_type(const Field& from_value, const IDataType& to_type, Field* to, + const IDataType* from_type_hint) { + if (from_value.is_null()) { + *to = from_value; + return Status::OK(); + } + if (from_type_hint && from_type_hint->equals(to_type)) { + *to = from_value; + return Status::OK(); + } + if (const auto* nullable_type = typeid_cast(&to_type)) { + const IDataType& nested_type = *nullable_type->get_nested_type(); + /// NULL remains NULL after any conversion. + if (WhichDataType(nested_type).is_nothing()) { + *to = {}; + return Status::OK(); + } + if (from_type_hint && from_type_hint->equals(nested_type)) { + *to = from_value; + return Status::OK(); + } + return convert_field_to_typeImpl(from_value, nested_type, from_type_hint, to); + } else { + return convert_field_to_typeImpl(from_value, to_type, from_type_hint, to); + } +} +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/data_types/convert_field_to_type.h b/be/src/vec/data_types/convert_field_to_type.h new file mode 100644 index 00000000000000..1f5567a139b114 --- /dev/null +++ b/be/src/vec/data_types/convert_field_to_type.h @@ -0,0 +1,37 @@ +// 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. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/Interpreters/convert_field_to_type.h +// and modified by Doris + +#pragma once +#include +namespace doris::vectorized { + +class IDataType; +/** Used to interpret expressions in a set in IN, + * and also in the query of the form INSERT ... VALUES ... + * + * To work correctly with expressions of the form `1.0 IN (1)` or, for example, `1 IN (1, 2.0, 2.5, -1)` work the same way as `1 IN (1, 2)`. + * Checks for the compatibility of types, checks values fall in the range of valid values of the type, makes type conversion. + * If the value does not fall into the range - returns Null. + */ + +Status convert_field_to_type(const Field& from_value, const IDataType& to_type, Field* field, + const IDataType* from_type_hint = nullptr); + +} // namespace doris::vectorized diff --git a/be/src/vec/data_types/data_type.cpp b/be/src/vec/data_types/data_type.cpp index 3fa53aa49c7d80..74c12118000493 100644 --- a/be/src/vec/data_types/data_type.cpp +++ b/be/src/vec/data_types/data_type.cpp @@ -137,6 +137,8 @@ PGenericType_TypeId IDataType::get_pdata_type(const IDataType* data_type) { return PGenericType::DATEV2; case TypeIndex::DateTime: return PGenericType::DATETIME; + case TypeIndex::VARIANT: + return PGenericType::VARIANT; case TypeIndex::DateTimeV2: return PGenericType::DATETIMEV2; case TypeIndex::BitMap: diff --git a/be/src/vec/data_types/data_type.h b/be/src/vec/data_types/data_type.h index e223f4a2e2446e..990901a4aa724d 100644 --- a/be/src/vec/data_types/data_type.h +++ b/be/src/vec/data_types/data_type.h @@ -177,6 +177,8 @@ class IDataType : private boost::noncopyable { */ virtual bool is_value_represented_by_integer() const { return false; } + virtual bool is_object() const { return false; } + /** Unsigned Integers, Date, DateTime. Not nullable. */ virtual bool is_value_represented_by_unsigned_integer() const { return false; } @@ -321,6 +323,8 @@ struct WhichDataType { bool is_nullable() const { return idx == TypeIndex::Nullable; } bool is_function() const { return idx == TypeIndex::Function; } bool is_aggregate_function() const { return idx == TypeIndex::AggregateFunction; } + bool is_variant_type() const { return idx == TypeIndex::VARIANT; } + bool is_simple() const { return is_int() || is_uint() || is_float() || is_string(); } }; /// IDataType helpers (alternative for IDataType virtual methods with single point of truth) diff --git a/be/src/vec/data_types/data_type_factory.cpp b/be/src/vec/data_types/data_type_factory.cpp index 6d26e02b8cb11e..65510dd9f2dcfb 100644 --- a/be/src/vec/data_types/data_type_factory.cpp +++ b/be/src/vec/data_types/data_type_factory.cpp @@ -21,6 +21,8 @@ #include "vec/data_types/data_type_factory.hpp" #include "data_type_time.h" +#include "vec/data_types/data_type_hll.h" +#include "vec/data_types/data_type_object.h" namespace doris::vectorized { @@ -186,6 +188,9 @@ DataTypePtr DataTypeFactory::create_data_type(const TypeDescriptor& col_desc, bo nested = std::make_shared(dataTypes, names); break; } + case TYPE_VARIANT: + // ColumnObject always none nullable + return std::make_shared("json", true); case INVALID_TYPE: default: DCHECK(false) << "invalid PrimitiveType:" << (int)col_desc.type; diff --git a/be/src/vec/data_types/data_type_object.cpp b/be/src/vec/data_types/data_type_object.cpp new file mode 100644 index 00000000000000..fe3ac5c9a263c8 --- /dev/null +++ b/be/src/vec/data_types/data_type_object.cpp @@ -0,0 +1,130 @@ +// 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. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/DataTypes/DataTypeObject.cpp +// and modified by Doris + +#include +#include +#include + +#include + +namespace doris::vectorized { + +DataTypeObject::DataTypeObject(const String& schema_format_, bool is_nullable_) + : schema_format(to_lower(schema_format_)), is_nullable(is_nullable_) {} +bool DataTypeObject::equals(const IDataType& rhs) const { + if (const auto* object = typeid_cast(&rhs)) { + return schema_format == object->schema_format && is_nullable == object->is_nullable; + } + return false; +} + +int64_t DataTypeObject::get_uncompressed_serialized_bytes(const IColumn& column, + int be_exec_version) const { + const auto& column_object = assert_cast(column); + assert(column_object.is_finalized()); + + const auto& subcolumns = column_object.get_subcolumns(); + size_t size = 0; + + size += sizeof(uint32_t); + for (const auto& entry : subcolumns) { + auto type = entry->data.get_least_common_type(); + + PColumnMeta column_meta_pb; + column_meta_pb.set_name(entry->path.get_path()); + type->to_pb_column_meta(&column_meta_pb); + std::string meta_binary; + column_meta_pb.SerializeToString(&meta_binary); + size += sizeof(uint32_t); + size += meta_binary.size(); + + size += type->get_uncompressed_serialized_bytes(entry->data.get_finalized_column(), + be_exec_version); + } + + return size; +} + +char* DataTypeObject::serialize(const IColumn& column, char* buf, int be_exec_version) const { + const auto& column_object = assert_cast(column); + assert(column_object.is_finalized()); + + const auto& subcolumns = column_object.get_subcolumns(); + + // 1. serialize num of subcolumns + *reinterpret_cast(buf) = subcolumns.size(); + buf += sizeof(uint32_t); + + // 2. serialize each subcolumn in a loop + for (const auto& entry : subcolumns) { + // 2.1 serialize subcolumn column meta pb (path and type) + auto type = entry->data.get_least_common_type(); + + PColumnMeta column_meta_pb; + column_meta_pb.set_name(entry->path.get_path()); + type->to_pb_column_meta(&column_meta_pb); + std::string meta_binary; + column_meta_pb.SerializeToString(&meta_binary); + *reinterpret_cast(buf) = meta_binary.size(); + buf += sizeof(uint32_t); + memcpy(buf, meta_binary.data(), meta_binary.size()); + buf += meta_binary.size(); + + // 2.2 serialize subcolumn + buf = type->serialize(entry->data.get_finalized_column(), buf, be_exec_version); + } + + return buf; +} + +const char* DataTypeObject::deserialize(const char* buf, IColumn* column, + int be_exec_version) const { + auto column_object = assert_cast(column); + + // 1. deserialize num of subcolumns + uint32_t num_subcolumns = *reinterpret_cast(buf); + buf += sizeof(uint32_t); + + // 2. deserialize each subcolumn in a loop + for (uint32_t i = 0; i < num_subcolumns; i++) { + // 2.1 deserialize subcolumn column path (str size + str data) + uint32_t size = *reinterpret_cast(buf); + buf += sizeof(uint32_t); + std::string meta_binary {buf, size}; + buf += size; + PColumnMeta column_meta_pb; + column_meta_pb.ParseFromString(meta_binary); + + // 2.2 deserialize subcolumn + auto type = DataTypeFactory::instance().create_data_type(column_meta_pb); + MutableColumnPtr sub_column = type->create_column(); + buf = type->deserialize(buf, sub_column.get(), be_exec_version); + + // add subcolumn to column_object + PathInData key {column_meta_pb.name()}; + column_object->add_sub_column(key, std::move(sub_column)); + } + + column_object->finalize(); + + return buf; +} + +} // namespace doris::vectorized diff --git a/be/src/vec/data_types/data_type_object.h b/be/src/vec/data_types/data_type_object.h new file mode 100644 index 00000000000000..2d8384bfd96acf --- /dev/null +++ b/be/src/vec/data_types/data_type_object.h @@ -0,0 +1,54 @@ +// 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. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/DataTypes/DataTypeObject.h +// and modified by Doris + +#pragma once +#include +#include +#include +namespace doris::vectorized { +class DataTypeObject : public IDataType { +private: + String schema_format; + bool is_nullable; + +public: + DataTypeObject(const String& schema_format_, bool is_nullable_); + const char* get_family_name() const override { return "Variant"; } + TypeIndex get_type_id() const override { return TypeIndex::VARIANT; } + MutableColumnPtr create_column() const override { return ColumnObject::create(is_nullable); } + bool is_object() const override { return true; } + bool equals(const IDataType& rhs) const override; + bool hasNullableSubcolumns() const { return is_nullable; } + bool get_is_parametric() const override { return true; } + bool can_be_inside_nullable() const override { return true; } + bool have_subtypes() const override { return true; }; + int64_t get_uncompressed_serialized_bytes(const IColumn& column, + int be_exec_version) const override; + std::string to_string(const IColumn& column, size_t row_num) const override { + const auto& column_object = assert_cast(column); + return "Variant: " + column_object.get_keys_str(); + } + char* serialize(const IColumn& column, char* buf, int be_exec_version) const override; + const char* deserialize(const char* buf, IColumn* column, int be_exec_version) const override; + [[noreturn]] Field get_default() const override { + LOG(FATAL) << "Method getDefault() is not implemented for data type " << get_name(); + } +}; +} // namespace doris::vectorized diff --git a/be/src/vec/data_types/get_least_supertype.cpp b/be/src/vec/data_types/get_least_supertype.cpp index 8c8b9d7e7ba7d6..9c0f00189f69ea 100644 --- a/be/src/vec/data_types/get_least_supertype.cpp +++ b/be/src/vec/data_types/get_least_supertype.cpp @@ -50,12 +50,152 @@ String get_exception_message_prefix(const DataTypes& types) { } } // namespace -DataTypePtr get_least_supertype(const DataTypes& types) { +Status get_numeric_type(const TypeIndexSet& types, DataTypePtr* type) { + bool all_numbers = true; + + size_t max_bits_of_signed_integer = 0; + size_t max_bits_of_unsigned_integer = 0; + size_t max_mantissa_bits_of_floating = 0; + + auto maximize = [](size_t& what, size_t value) { + if (value > what) what = value; + }; + + for (const auto& type : types) { + if (type == TypeIndex::UInt8) + maximize(max_bits_of_unsigned_integer, 8); + else if (type == TypeIndex::UInt16) + maximize(max_bits_of_unsigned_integer, 16); + else if (type == TypeIndex::UInt32) + maximize(max_bits_of_unsigned_integer, 32); + else if (type == TypeIndex::UInt64) + maximize(max_bits_of_unsigned_integer, 64); + else if (type == TypeIndex::UInt128) + maximize(max_bits_of_unsigned_integer, 128); + else if (type == TypeIndex::Int8 || type == TypeIndex::Enum8) + maximize(max_bits_of_signed_integer, 8); + else if (type == TypeIndex::Int16 || type == TypeIndex::Enum16) + maximize(max_bits_of_signed_integer, 16); + else if (type == TypeIndex::Int32) + maximize(max_bits_of_signed_integer, 32); + else if (type == TypeIndex::Int64) + maximize(max_bits_of_signed_integer, 64); + else if (type == TypeIndex::Int128) + maximize(max_bits_of_signed_integer, 128); + else if (type == TypeIndex::Float32) + maximize(max_mantissa_bits_of_floating, 24); + else if (type == TypeIndex::Float64) + maximize(max_mantissa_bits_of_floating, 53); + else + all_numbers = false; + } + + if (max_bits_of_signed_integer || max_bits_of_unsigned_integer || + max_mantissa_bits_of_floating) { + if (!all_numbers) { + LOG(INFO) << " because some of them are numbers and some of them are not"; + *type = nullptr; + return Status::InvalidArgument("some of them are numbers and some of them are not"); + } + + /// If there are signed and unsigned types of same bit-width, the result must be signed number with at least one more bit. + /// Example, common of Int32, UInt32 = Int64. + + size_t min_bit_width_of_integer = + std::max(max_bits_of_signed_integer, max_bits_of_unsigned_integer); + + /// If unsigned is not covered by signed. + if (max_bits_of_signed_integer && + max_bits_of_unsigned_integer >= max_bits_of_signed_integer) + ++min_bit_width_of_integer; + + /// If the result must be floating. + if (max_mantissa_bits_of_floating) { + size_t min_mantissa_bits = + std::max(min_bit_width_of_integer, max_mantissa_bits_of_floating); + if (min_mantissa_bits <= 24) { + *type = std::make_shared(); + return Status::OK(); + } else if (min_mantissa_bits <= 53) { + *type = std::make_shared(); + return Status::OK(); + } else { + LOG(INFO) << " because some of them are integers and some are floating point " + "but there is no floating point type, that can exactly represent " + "all required integers"; + *type = nullptr; + return Status::InvalidArgument( + "there is no floating point type, that can exactly represent " + "all required integers"); + } + } + + /// If the result must be signed integer. + if (max_bits_of_signed_integer) { + if (min_bit_width_of_integer <= 8) { + *type = std::make_shared(); + return Status::OK(); + } else if (min_bit_width_of_integer <= 16) { + *type = std::make_shared(); + return Status::OK(); + } else if (min_bit_width_of_integer <= 32) { + *type = std::make_shared(); + return Status::OK(); + } else if (min_bit_width_of_integer <= 64) { + *type = std::make_shared(); + return Status::OK(); + } else { + LOG(INFO) << " because some of them are signed integers and some are unsigned " + "integers, but there is no signed integer type, that can exactly " + "represent all required unsigned integer values"; + return Status::InvalidArgument( + "there is no signed integer type, that can exactly " + "represent all required unsigned integer values"); + } + } + + /// All unsigned. + { + if (min_bit_width_of_integer <= 8) { + *type = std::make_shared(); + return Status::OK(); + } else if (min_bit_width_of_integer <= 16) { + *type = std::make_shared(); + return Status::OK(); + } else if (min_bit_width_of_integer <= 32) { + *type = std::make_shared(); + return Status::OK(); + } else if (min_bit_width_of_integer <= 64) { + *type = std::make_shared(); + return Status::OK(); + } else { + LOG(FATAL) << "Logical error: " + << "but as all data types are unsigned integers, we must have found " + "maximum unsigned integer type"; + *type = nullptr; + return Status::InvalidArgument( + "all data types are unsigned integers, we must have found " + "maximum unsigned integer type"); + } + } + } + *type = nullptr; + return Status::OK(); +} + +// TODO conflict type resolve +Status get_least_supertype(const DataTypes& types, DataTypePtr* type, bool compatible_with_string) { /// Trivial cases - if (types.empty()) return std::make_shared(); + if (types.empty()) { + *type = std::make_shared(); + return Status::OK(); + } - if (types.size() == 1) return types[0]; + if (types.size() == 1) { + *type = types[0]; + return Status::OK(); + } /// All types are equal { @@ -67,7 +207,10 @@ DataTypePtr get_least_supertype(const DataTypes& types) { } } - if (all_equal) return types[0]; + if (all_equal) { + *type = types[0]; + return Status::OK(); + } } /// Recursive rules @@ -81,7 +224,8 @@ DataTypePtr get_least_supertype(const DataTypes& types) { if (!typeid_cast(type.get())) non_nothing_types.emplace_back(type); - if (non_nothing_types.size() < types.size()) return get_least_supertype(non_nothing_types); + if (non_nothing_types.size() < types.size()) + return get_least_supertype(non_nothing_types, type, compatible_with_string); } /// For Nullable @@ -103,13 +247,19 @@ DataTypePtr get_least_supertype(const DataTypes& types) { } if (have_nullable) { - return std::make_shared(get_least_supertype(nested_types)); + DataTypePtr nested_type; + Status st = get_least_supertype(nested_types, &nested_type, compatible_with_string); + if (!st.ok()) { + return st; + } + *type = std::make_shared(nested_type); + return st; } } /// Non-recursive rules - std::unordered_set type_ids; + phmap::flat_hash_set type_ids; for (const auto& type : types) type_ids.insert(type->get_type_id()); /// For String and FixedString, or for different FixedStrings, the common type is String. @@ -120,13 +270,16 @@ DataTypePtr get_least_supertype(const DataTypes& types) { if (have_string || have_fixed_string) { bool all_strings = type_ids.size() == (have_string + have_fixed_string); - if (!all_strings) { - LOG(FATAL) + if (!all_strings && !compatible_with_string) { + LOG(INFO) << get_exception_message_prefix(types) << " because some of them are String/FixedString and some of them are not"; + return Status::InvalidArgument( + "some of them are String/FixedString and some of them are not"); } - return std::make_shared(); + *type = std::make_shared(); + return Status::OK(); } } @@ -138,28 +291,14 @@ DataTypePtr get_least_supertype(const DataTypes& types) { if (have_date || have_datetime) { bool all_date_or_datetime = type_ids.size() == (have_date + have_datetime); if (!all_date_or_datetime) { - LOG(FATAL) << get_exception_message_prefix(types) - << " because some of them are Date/DateTime and some of them are not"; + LOG(INFO) << get_exception_message_prefix(types) + << " because some of them are Date/DateTime and some of them are not"; + return Status::InvalidArgument( + "because some of them are Date/DateTime and some of them are not"); } - return std::make_shared(); - } - } - - { - UInt32 have_date_v2 = type_ids.count(TypeIndex::DateV2); - - UInt32 have_datetime_v2 = type_ids.count(TypeIndex::DateTimeV2); - - if (have_date_v2 || have_datetime_v2) { - bool all_datev2_or_datetimev2 = type_ids.size() == (have_date_v2 + have_datetime_v2); - if (!all_datev2_or_datetimev2) { - LOG(FATAL) - << get_exception_message_prefix(types) - << " because some of them are DateV2/DateTimeV2 and some of them are not"; - } - - return std::make_shared(); + *type = std::make_shared(); + return Status::OK(); } } @@ -188,8 +327,10 @@ DataTypePtr get_least_supertype(const DataTypes& types) { } if (num_supported != type_ids.size()) { - LOG(FATAL) << get_exception_message_prefix(types) - << " because some of them have no lossless convertion to Decimal"; + LOG(INFO) << get_exception_message_prefix(types) + << " because some of them have no lossless convertion to Decimal"; + return Status::InvalidArgument( + "some of them have no lossless convertion to Decimal"); } UInt32 max_scale = 0; @@ -209,136 +350,95 @@ DataTypePtr get_least_supertype(const DataTypes& types) { } if (min_precision > DataTypeDecimal::max_precision()) { - LOG(FATAL) << fmt::format("{} because the least supertype is Decimal({},{})", - get_exception_message_prefix(types), min_precision, - max_scale); + LOG(INFO) << fmt::format("{} because the least supertype is Decimal({},{})", + get_exception_message_prefix(types), min_precision, + max_scale); + return Status::InvalidArgument( + fmt::format("{} because the least supertype is Decimal({},{})", + get_exception_message_prefix(types), min_precision, max_scale)); } - if (have_decimal128 || min_precision > DataTypeDecimal::max_precision()) - return std::make_shared>( + if (have_decimal128 || min_precision > DataTypeDecimal::max_precision()) { + *type = std::make_shared>( DataTypeDecimal::max_precision(), max_scale); - if (have_decimal128i || min_precision > DataTypeDecimal::max_precision()) - return std::make_shared>( + return Status::OK(); + } + if (have_decimal128i || min_precision > DataTypeDecimal::max_precision()) { + *type = std::make_shared>( DataTypeDecimal::max_precision(), max_scale); - if (have_decimal64 || min_precision > DataTypeDecimal::max_precision()) - return std::make_shared>( + return Status::OK(); + } + if (have_decimal64 || min_precision > DataTypeDecimal::max_precision()) { + *type = std::make_shared>( DataTypeDecimal::max_precision(), max_scale); - return std::make_shared>( + return Status::OK(); + } + *type = std::make_shared>( DataTypeDecimal::max_precision(), max_scale); + return Status::OK(); } } /// For numeric types, the most complicated part. { - bool all_numbers = true; - - size_t max_bits_of_signed_integer = 0; - size_t max_bits_of_unsigned_integer = 0; - size_t max_mantissa_bits_of_floating = 0; + DataTypePtr numeric_type = nullptr; + Status st = get_numeric_type(type_ids, &numeric_type); + if (numeric_type) { + DCHECK(st.ok()); + *type = numeric_type; + return Status::OK(); + } + } - auto maximize = [](size_t& what, size_t value) { - if (value > what) what = value; - }; + /// All other data types (UUID, AggregateFunction, Enum...) are compatible only if they are the same (checked in trivial cases). + *type = nullptr; + return Status::InvalidArgument(get_exception_message_prefix(types)); +} - for (const auto& type : types) { - if (typeid_cast(type.get())) - maximize(max_bits_of_unsigned_integer, 8); - else if (typeid_cast(type.get())) - maximize(max_bits_of_unsigned_integer, 16); - else if (typeid_cast(type.get())) - maximize(max_bits_of_unsigned_integer, 32); - else if (typeid_cast(type.get())) - maximize(max_bits_of_unsigned_integer, 64); - else if (typeid_cast(type.get())) - maximize(max_bits_of_signed_integer, 8); - else if (typeid_cast(type.get())) - maximize(max_bits_of_signed_integer, 16); - else if (typeid_cast(type.get())) - maximize(max_bits_of_signed_integer, 32); - else if (typeid_cast(type.get())) - maximize(max_bits_of_signed_integer, 64); - else if (typeid_cast(type.get())) - maximize(max_mantissa_bits_of_floating, 24); - else if (typeid_cast(type.get())) - maximize(max_mantissa_bits_of_floating, 53); - else - all_numbers = false; +Status get_least_supertype(const TypeIndexSet& types, DataTypePtr* type, + bool compatible_with_string) { + TypeIndexSet types_set; + for (const auto& t : types) { + if (WhichDataType(t).is_nothing()) continue; + + if (!WhichDataType(t).is_simple()) { + LOG(INFO) << "Cannot get common type by type ids with parametric type" + << getTypeName(t); + *type = nullptr; + return Status::InvalidArgument( + "Cannot get common type by type ids with parametric type"); } - if (max_bits_of_signed_integer || max_bits_of_unsigned_integer || - max_mantissa_bits_of_floating) { - if (!all_numbers) { - LOG(FATAL) << get_exception_message_prefix(types) - << " because some of them are numbers and some of them are not"; - } - - /// If there are signed and unsigned types of same bit-width, the result must be signed number with at least one more bit. - /// Example, common of Int32, UInt32 = Int64. - - size_t min_bit_width_of_integer = - std::max(max_bits_of_signed_integer, max_bits_of_unsigned_integer); - - /// If unsigned is not covered by signed. - if (max_bits_of_signed_integer && - max_bits_of_unsigned_integer >= max_bits_of_signed_integer) - ++min_bit_width_of_integer; - - /// If the result must be floating. - if (max_mantissa_bits_of_floating) { - size_t min_mantissa_bits = - std::max(min_bit_width_of_integer, max_mantissa_bits_of_floating); - if (min_mantissa_bits <= 24) - return std::make_shared(); - else if (min_mantissa_bits <= 53) - return std::make_shared(); - else { - LOG(FATAL) << get_exception_message_prefix(types) - << " because some of them are integers and some are floating point " - "but there is no floating point type, that can exactly represent " - "all required integers"; - } - } + types_set.insert(t); + } - /// If the result must be signed integer. - if (max_bits_of_signed_integer) { - if (min_bit_width_of_integer <= 8) - return std::make_shared(); - else if (min_bit_width_of_integer <= 16) - return std::make_shared(); - else if (min_bit_width_of_integer <= 32) - return std::make_shared(); - else if (min_bit_width_of_integer <= 64) - return std::make_shared(); - else { - LOG(FATAL) << get_exception_message_prefix(types) - << " because some of them are signed integers and some are unsigned " - "integers, but there is no signed integer type, that can exactly " - "represent all required unsigned integer values"; - } - } + if (types_set.empty()) { + *type = std::make_shared(); + return Status::OK(); + } - /// All unsigned. - { - if (min_bit_width_of_integer <= 8) - return std::make_shared(); - else if (min_bit_width_of_integer <= 16) - return std::make_shared(); - else if (min_bit_width_of_integer <= 32) - return std::make_shared(); - else if (min_bit_width_of_integer <= 64) - return std::make_shared(); - else { - LOG(FATAL) << "Logical error: " << get_exception_message_prefix(types) - << "but as all data types are unsigned integers, we must have found " - "maximum unsigned integer type"; - } - } + if (types.count(TypeIndex::String)) { + if (types.size() != 1 && !compatible_with_string) { + LOG(INFO) << " because some of them are String and some of them are not"; + *type = nullptr; + return Status::InvalidArgument("some of them are String and some of them are not"); } + + *type = std::make_shared(); + return Status::OK(); } + /// For numeric types, the most complicated part. + DataTypePtr numeric_type = nullptr; + Status st = get_numeric_type(types, &numeric_type); + if (numeric_type) { + DCHECK(st.ok()); + *type = numeric_type; + return Status::OK(); + } /// All other data types (UUID, AggregateFunction, Enum...) are compatible only if they are the same (checked in trivial cases). - LOG(FATAL) << get_exception_message_prefix(types); - return nullptr; + *type = nullptr; + return Status::InvalidArgument("unknown type"); } } // namespace doris::vectorized diff --git a/be/src/vec/data_types/get_least_supertype.h b/be/src/vec/data_types/get_least_supertype.h index 64f6e7619a5711..e0ab9438034205 100644 --- a/be/src/vec/data_types/get_least_supertype.h +++ b/be/src/vec/data_types/get_least_supertype.h @@ -20,8 +20,11 @@ #pragma once +#include #include +#include "common/status.h" + namespace doris::vectorized { /** Get data type that covers all possible values of passed data types. @@ -30,6 +33,13 @@ namespace doris::vectorized { * Examples: least common supertype for UInt8, Int8 - Int16. * Examples: there is no least common supertype for Array(UInt8), Int8. */ -DataTypePtr get_least_supertype(const DataTypes& types); + +using TypeIndexSet = phmap::flat_hash_set; + +Status get_least_supertype(const DataTypes& types, DataTypePtr* type, + bool compatible_with_string = false); + +Status get_least_supertype(const TypeIndexSet& types, DataTypePtr* type, + bool compatible_with_string = false); } // namespace doris::vectorized diff --git a/be/src/vec/exec/format/json/new_json_reader.cpp b/be/src/vec/exec/format/json/new_json_reader.cpp index 57cb70f2de037c..ce6d3716589a87 100644 --- a/be/src/vec/exec/format/json/new_json_reader.cpp +++ b/be/src/vec/exec/format/json/new_json_reader.cpp @@ -27,13 +27,21 @@ #include "vec/core/block.h" #include "vec/exec/format/file_reader/new_plain_text_line_reader.h" #include "vec/exec/scan/vscanner.h" +#include "vec/json/simd_json_parser.h" + +// dynamic table +#include "vec/columns/column_object.h" +#include "vec/common/schema_util.h" +#include "vec/json/json_parser.h" +#include "vec/json/parse2column.h" + namespace doris::vectorized { using namespace ErrorCode; NewJsonReader::NewJsonReader(RuntimeState* state, RuntimeProfile* profile, ScannerCounter* counter, const TFileScanRangeParams& params, const TFileRangeDesc& range, const std::vector& file_slot_descs, bool* scanner_eof, - IOContext* io_ctx) + IOContext* io_ctx, bool is_dynamic_schema) : _vhandle_json_callback(nullptr), _state(state), _profile(profile), @@ -53,7 +61,8 @@ NewJsonReader::NewJsonReader(RuntimeState* state, RuntimeProfile* profile, Scann _origin_json_doc(&_value_allocator, sizeof(_parse_buffer), &_parse_allocator), _scanner_eof(scanner_eof), _current_offset(0), - _io_ctx(io_ctx) { + _io_ctx(io_ctx), + _is_dynamic_schema(is_dynamic_schema) { _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES); _read_timer = ADD_TIMER(_profile, "ReadTime"); _file_read_timer = ADD_TIMER(_profile, "FileReadTime"); @@ -91,7 +100,8 @@ Status NewJsonReader::init_reader() { //improve performance if (_parsed_jsonpaths.empty()) { // input is a simple json-string - _vhandle_json_callback = &NewJsonReader::_vhandle_simple_json; + _vhandle_json_callback = _is_dynamic_schema ? &NewJsonReader::_vhandle_dynamic_json + : &NewJsonReader::_vhandle_simple_json; } else { // input is a complex json-string and a json-path if (_strip_outer_array) { _vhandle_json_callback = &NewJsonReader::_vhandle_flat_array_complex_json; @@ -99,6 +109,9 @@ Status NewJsonReader::init_reader() { _vhandle_json_callback = &NewJsonReader::_vhandle_nested_complex_json; } } + if (_is_dynamic_schema) { + _json_parser = std::make_unique>(); + } return Status::OK(); } @@ -111,7 +124,7 @@ Status NewJsonReader::get_next_block(Block* block, size_t* read_rows, bool* eof) const int batch_size = std::max(_state->batch_size(), (int)_MIN_BATCH_SIZE); auto columns = block->mutate_columns(); - while (columns[0]->size() < batch_size && !_reader_eof) { + while (columns.back()->size() < batch_size && !_reader_eof) { if (UNLIKELY(_read_json_by_line && _skip_first_line)) { size_t size = 0; const uint8_t* line_ptr = nullptr; @@ -365,6 +378,84 @@ Status NewJsonReader::_read_json_column(std::vector& columns, return (this->*_vhandle_json_callback)(columns, slot_descs, is_empty_row, eof); } +Status NewJsonReader::_parse_dynamic_json(bool* is_empty_row, bool* eof, + MutableColumnPtr& dynamic_column) { + size_t size = 0; + // read a whole message + SCOPED_TIMER(_file_read_timer); + const uint8_t* json_str = nullptr; + std::unique_ptr json_str_ptr; + if (_line_reader != nullptr) { + RETURN_IF_ERROR(_line_reader->read_line(&json_str, &size, eof)); + } else { + size_t length = 0; + RETURN_IF_ERROR(_read_one_message(&json_str_ptr, &length)); + json_str = json_str_ptr.get(); + size = length; + if (length == 0) { + *eof = true; + } else if (json_str[size] == '\n') { + size--; + } + } + + _bytes_read_counter += size; + + // read all data, then return + if (size == 0 || *eof) { + *is_empty_row = true; + return Status::OK(); + } + + auto& column_object = assert_cast(*(dynamic_column.get())); + Status st = doris::vectorized::parse_json_to_variant(column_object, StringRef {json_str, size}, + _json_parser.get()); + if (st.is()) { + fmt::memory_buffer error_msg; + fmt::format_to(error_msg, "Parse json data for JsonDoc failed. error info: {}", + st.to_string()); + RETURN_IF_ERROR(_state->append_error_msg_to_file( + [&]() -> std::string { return std::string((char*)json_str, size); }, + [&]() -> std::string { return fmt::to_string(error_msg); }, _scanner_eof)); + _counter->num_rows_filtered++; + if (*_scanner_eof) { + // Case A: if _scanner_eof is set to true in "append_error_msg_to_file", which means + // we meet enough invalid rows and the scanner should be stopped. + // So we set eof to true and return OK, the caller will stop the process as we meet the end of file. + *eof = true; + return Status::OK(); + } + return Status::DataQualityError(fmt::to_string(error_msg)); + } + + if (_strip_outer_array) { + column_object.finalize(); + column_object.strip_outer_array(); + } + + return Status::OK(); +} + +Status NewJsonReader::_vhandle_dynamic_json(std::vector& columns, + const std::vector& slot_descs, + bool* is_empty_row, bool* eof) { + MutableColumnPtr& dynamic_column = columns.back(); + bool valid = false; + do { + Status st = _parse_dynamic_json(is_empty_row, eof, dynamic_column); + if (st.is()) { + continue; // continue to read next + } + RETURN_IF_ERROR(st); + if (*is_empty_row == true) { + return Status::OK(); + } + *is_empty_row = false; + valid = true; + } while (!valid); + return Status::OK(); +} + Status NewJsonReader::_vhandle_simple_json(std::vector& columns, const std::vector& slot_descs, bool* is_empty_row, bool* eof) { diff --git a/be/src/vec/exec/format/json/new_json_reader.h b/be/src/vec/exec/format/json/new_json_reader.h index b67f9393e13450..0c14342204a1c6 100644 --- a/be/src/vec/exec/format/json/new_json_reader.h +++ b/be/src/vec/exec/format/json/new_json_reader.h @@ -25,6 +25,7 @@ #include "io/fs/file_reader.h" #include "olap/iterators.h" #include "vec/exec/format/generic_reader.h" + namespace doris { class FileReader; @@ -35,13 +36,16 @@ class SlotDescriptor; namespace vectorized { struct ScannerCounter; +template +class JSONDataParser; +class SimdJSONParser; class NewJsonReader : public GenericReader { public: NewJsonReader(RuntimeState* state, RuntimeProfile* profile, ScannerCounter* counter, const TFileScanRangeParams& params, const TFileRangeDesc& range, const std::vector& file_slot_descs, bool* scanner_eof, - IOContext* io_ctx); + IOContext* io_ctx, bool is_dynamic_schema = false); NewJsonReader(RuntimeProfile* profile, const TFileScanRangeParams& params, const TFileRangeDesc& range, const std::vector& file_slot_descs, @@ -69,6 +73,11 @@ class NewJsonReader : public GenericReader { const std::vector& slot_descs, bool* is_empty_row, bool* eof); + Status _parse_dynamic_json(bool* is_empty_row, bool* eof, MutableColumnPtr& dynamic_column); + Status _vhandle_dynamic_json(std::vector& columns, + const std::vector& slot_descs, bool* is_empty_row, + bool* eof); + Status _vhandle_flat_array_complex_json(std::vector& columns, const std::vector& slot_descs, bool* is_empty_row, bool* eof); @@ -152,6 +161,9 @@ class NewJsonReader : public GenericReader { RuntimeProfile::Counter* _bytes_read_counter; RuntimeProfile::Counter* _read_timer; RuntimeProfile::Counter* _file_read_timer; + + bool _is_dynamic_schema = false; + std::unique_ptr> _json_parser; }; } // namespace vectorized } // namespace doris diff --git a/be/src/vec/exec/scan/scanner_scheduler.cpp b/be/src/vec/exec/scan/scanner_scheduler.cpp index 085d64726290d0..fb5da32548cdc5 100644 --- a/be/src/vec/exec/scan/scanner_scheduler.cpp +++ b/be/src/vec/exec/scan/scanner_scheduler.cpp @@ -325,7 +325,11 @@ void ScannerScheduler::_scanner_scan(ScannerScheduler* scheduler, ScannerContext if (UNLIKELY(block->rows() == 0)) { ctx->return_free_block(block); } else { - if (!blocks.empty() && blocks.back()->rows() + block->rows() <= state->batch_size()) { + if (!blocks.empty() && + blocks.back()->rows() + block->rows() <= state->batch_size() + // block may miss match bettween dynamic blocks + // merge is not supported by dynamic block + && blocks.back()->get_block_type() != BlockType::DYNAMIC) { vectorized::MutableBlock(blocks.back()).merge(*block); ctx->return_free_block(block); } else { diff --git a/be/src/vec/exec/scan/vfile_scanner.cpp b/be/src/vec/exec/scan/vfile_scanner.cpp index 0bb61e54c5a356..1b7dee51886915 100644 --- a/be/src/vec/exec/scan/vfile_scanner.cpp +++ b/be/src/vec/exec/scan/vfile_scanner.cpp @@ -113,6 +113,30 @@ Status VFileScanner::open(RuntimeState* state) { return Status::OK(); } +Status VFileScanner::_handle_dynamic_block(Block* block) { + // finalize object column + auto obj = assert_cast(block->get_columns().back().get()); + const_cast(obj)->finalize(); + + // flatten object columns for the purpose of extracting static columns and + // fill default values missing in static columns + RETURN_IF_ERROR(schema_util::flatten_object(*block, true /*replace static columns*/)); + + bool need_issue_rpc = false; + for (const auto& name : block->get_names()) { + // missing column in base schema + if (!_full_base_schema_view->column_name_to_column.contains(name)) { + need_issue_rpc = true; + } + } + if (need_issue_rpc) { + // duplicated columns in _full_base_schema_view will be idempotent + RETURN_IF_ERROR(vectorized::schema_util::send_add_columns_rpc( + block->get_columns_with_type_and_name(), _full_base_schema_view.get())); + } + return Status::OK(); +} + // For query: // [exist cols] [non-exist cols] [col from path] input output // A B C D E @@ -155,6 +179,10 @@ Status VFileScanner::_get_block_impl(RuntimeState* state, Block* block, bool* eo _cur_reader->get_next_block(_src_block_ptr, &read_rows, &_cur_reader_eof)); } + if (_is_dynamic_schema) { + RETURN_IF_ERROR(_handle_dynamic_block(_src_block_ptr)); + } + // use read_rows instead of _src_block_ptr->rows(), because the first column of _src_block_ptr // may not be filled after calling `get_next_block()`, so _src_block_ptr->rows() may return wrong result. if (read_rows > 0) { @@ -207,7 +235,7 @@ Status VFileScanner::_init_src_block(Block* block) { for (auto& slot : _input_tuple_desc->slots()) { DataTypePtr data_type; auto it = _name_to_col_type.find(slot->col_name()); - if (it == _name_to_col_type.end()) { + if (it == _name_to_col_type.end() || _is_dynamic_schema) { // not exist in file, using type from _input_tuple_desc data_type = DataTypeFactory::instance().create_data_type(slot->type(), slot->is_nullable()); @@ -234,6 +262,11 @@ Status VFileScanner::_cast_to_input_block(Block* block) { if (!_is_load) { return Status::OK(); } + if (_is_dynamic_schema) { + // Dynamic schema do not need cast from string, + // since it's already casted in ColumnObject + return Status::OK(); + } SCOPED_TIMER(_cast_to_input_block_timer); // cast primitive type(PT0) to primitive type(PT1) size_t idx = 0; @@ -376,21 +409,44 @@ Status VFileScanner::_convert_to_output_block(Block* block) { auto& filter_map = filter_column->get_data(); auto origin_column_num = _src_block.columns(); + // Set block dynamic, block maybe merge or add_rows + // in in later process. + if (_is_dynamic_schema) { + block->set_block_type(BlockType::DYNAMIC); + } + for (auto slot_desc : _output_tuple_desc->slots()) { if (!slot_desc->is_materialized()) { continue; } int dest_index = ctx_idx++; - auto* ctx = _dest_vexpr_ctx[dest_index]; - int result_column_id = -1; - // PT1 => dest primitive type - RETURN_IF_ERROR(ctx->execute(&_src_block, &result_column_id)); - bool is_origin_column = result_column_id < origin_column_num; - auto column_ptr = - is_origin_column && _src_block_mem_reuse - ? _src_block.get_by_position(result_column_id).column->clone_resized(rows) - : _src_block.get_by_position(result_column_id).column; + vectorized::ColumnPtr column_ptr; + if (_is_dynamic_schema) { + if (slot_desc->type().is_variant_type()) { + continue; + } + // cast column + auto& column_type_name = _src_block.get_by_position(dest_index); + auto dest_type = vectorized::DataTypeFactory::instance().create_data_type( + slot_desc->type(), slot_desc->is_nullable()); + if (!column_type_name.type->equals(*dest_type)) { + RETURN_IF_ERROR(vectorized::schema_util::cast_column(column_type_name, dest_type, + &column_ptr)); + } else { + column_ptr = std::move(column_type_name.column); + } + } else { + auto* ctx = _dest_vexpr_ctx[dest_index]; + int result_column_id = -1; + // PT1 => dest primitive type + RETURN_IF_ERROR(ctx->execute(&_src_block, &result_column_id)); + bool is_origin_column = result_column_id < origin_column_num; + column_ptr = is_origin_column && _src_block_mem_reuse + ? _src_block.get_by_position(result_column_id) + .column->clone_resized(rows) + : _src_block.get_by_position(result_column_id).column; + } // column_ptr maybe a ColumnConst, convert it to a normal column column_ptr = column_ptr->convert_to_full_column_if_const(); @@ -454,6 +510,32 @@ Status VFileScanner::_convert_to_output_block(Block* block) { slot_desc->col_name())); } + // handle dynamic generated columns + if (_full_base_schema_view && !_full_base_schema_view->empty()) { + CHECK(_is_dynamic_schema); + for (size_t i = block->columns(); i < _src_block.columns(); ++i) { + auto& column_type_name = _src_block.get_by_position(i); + // Column from schema change response + const TColumn& tcolumn = + _full_base_schema_view->column_name_to_column[column_type_name.name]; + auto original_type = vectorized::DataTypeFactory::instance().create_data_type(tcolumn); + // Detect type conflict, there may exist another load procedure, whitch has already added some columns + // but, this load detects different type, we go type conflict free path, always cast to original type + // TODO need to add type conflict abort feature + if (!column_type_name.type->equals(*original_type)) { + vectorized::ColumnPtr column_ptr; + RETURN_IF_ERROR(vectorized::schema_util::cast_column(column_type_name, + original_type, &column_ptr)); + column_type_name.column = column_ptr; + column_type_name.type = original_type; + } + DCHECK(column_type_name.column != nullptr); + block->insert(vectorized::ColumnWithTypeAndName(std::move(column_type_name.column), + std::move(column_type_name.type), + column_type_name.name)); + } + } + // after do the dest block insert operation, clear _src_block to remove the reference of origin column if (_src_block_mem_reuse) { _src_block.clear_column_data(origin_column_num); @@ -467,6 +549,7 @@ Status VFileScanner::_convert_to_output_block(Block* block) { std::make_shared(), "filter column")); RETURN_IF_ERROR(vectorized::Block::filter_block(block, dest_size, dest_size)); + _counter.num_rows_filtered += rows - block->rows(); return Status::OK(); } @@ -533,7 +616,8 @@ Status VFileScanner::_get_next_reader() { } case TFileFormatType::FORMAT_JSON: { _cur_reader.reset(new NewJsonReader(_state, _profile, &_counter, _params, range, - _file_slot_descs, &_scanner_eof, _io_ctx.get())); + _file_slot_descs, &_scanner_eof, _io_ctx.get(), + _is_dynamic_schema)); init_status = ((NewJsonReader*)(_cur_reader.get()))->init_reader(); break; } @@ -725,6 +809,17 @@ Status VFileScanner::_init_expr_ctxes() { } } } + // If last slot is_variant from stream plan which indicate table is dynamic schema + _is_dynamic_schema = + _output_tuple_desc && _output_tuple_desc->slots().back()->type().is_variant_type(); + if (_is_dynamic_schema) { + // should not resuse Block since Block is variable + _src_block_mem_reuse = false; + _full_base_schema_view.reset(new vectorized::schema_util::FullBaseSchemaView); + _full_base_schema_view->db_name = _output_tuple_desc->table_desc()->database(); + _full_base_schema_view->table_name = _output_tuple_desc->table_desc()->name(); + _full_base_schema_view->table_id = _output_tuple_desc->table_desc()->table_id(); + } return Status::OK(); } diff --git a/be/src/vec/exec/scan/vfile_scanner.h b/be/src/vec/exec/scan/vfile_scanner.h index 9a00e10dab4780..2ca6f2a6d5d7e8 100644 --- a/be/src/vec/exec/scan/vfile_scanner.h +++ b/be/src/vec/exec/scan/vfile_scanner.h @@ -21,6 +21,7 @@ #include "exec/text_converter.h" #include "exprs/function_filter.h" #include "io/file_factory.h" +#include "vec/common/schema_util.h" #include "vec/exec/format/format_common.h" #include "vec/exec/format/generic_reader.h" #include "vec/exec/scan/vscanner.h" @@ -117,6 +118,9 @@ class VFileScanner : public VScanner { Block _src_block; VExprContext* _push_down_expr = nullptr; + bool _is_dynamic_schema = false; + // for tracing dynamic schema + std::unique_ptr _full_base_schema_view; std::unique_ptr _file_cache_statistics; std::unique_ptr _io_ctx; @@ -138,6 +142,7 @@ class VFileScanner : public VScanner { Status _pre_filter_src_block(); Status _convert_to_output_block(Block* block); Status _generate_fill_columns(); + Status _handle_dynamic_block(Block* block); void _reset_counter() { _counter.num_rows_unselected = 0; diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h index b87e0142c6f504..c8dc4282df7519 100644 --- a/be/src/vec/functions/function_cast.h +++ b/be/src/vec/functions/function_cast.h @@ -1765,12 +1765,16 @@ class FunctionBuilderCast : public FunctionBuilderImpl { DataTypePtr get_return_type_impl(const ColumnsWithTypeAndName& arguments) const override { const auto type_col = check_and_get_column_const(arguments.back().column.get()); + DataTypePtr type; if (!type_col) { - LOG(FATAL) << fmt::format( - "Second argument to {} must be a constant string describing type", get_name()); + // only used in schema_util::cast_column + // use second arg as type arg + // since not all types are in the DatatypeFactory + type = arguments[1].type; + } else { + // TODO(xy): support return struct type for factory + type = DataTypeFactory::instance().get(type_col->get_value()); } - // TODO(xy): support return struct type for factory - auto type = DataTypeFactory::instance().get(type_col->get_value()); DCHECK(type != nullptr); bool need_to_be_nullable = false; // 1. from_type is nullable diff --git a/be/src/vec/functions/if.cpp b/be/src/vec/functions/if.cpp index 031ec30118b5c3..a8eec9233eac8d 100644 --- a/be/src/vec/functions/if.cpp +++ b/be/src/vec/functions/if.cpp @@ -122,7 +122,10 @@ class FunctionIf : public IFunction { } DataTypePtr get_return_type_impl(const DataTypes& arguments) const override { - return get_least_supertype({arguments[1], arguments[2]}); + DataTypePtr type = nullptr; + get_least_supertype(DataTypes {arguments[1], arguments[2]}, &type); + DCHECK_NE(type, nullptr); + return type; } static ColumnPtr materialize_column_if_const(const ColumnPtr& column) { diff --git a/be/src/vec/io/var_int.h b/be/src/vec/io/var_int.h index e932c506b7177f..af979194e8c0c2 100644 --- a/be/src/vec/io/var_int.h +++ b/be/src/vec/io/var_int.h @@ -19,6 +19,7 @@ #include +#include "vec/common/string_buffer.hpp" #include "vec/core/types.h" namespace doris::vectorized { diff --git a/be/src/vec/json/json_parser.cpp b/be/src/vec/json/json_parser.cpp new file mode 100644 index 00000000000000..893634c19e8425 --- /dev/null +++ b/be/src/vec/json/json_parser.cpp @@ -0,0 +1,265 @@ +// 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. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/JSONParsers/SimdJSONParser.cpp +// and modified by Doris + +#include "vec/json/json_parser.h" + +#include "vec/json/simd_json_parser.h" + +namespace doris::vectorized { + +template +bool JSONDataParser::extract_key(MutableColumns& columns, StringRef json, + const std::vector& keys, + const std::vector& types) { + assert(types.size() == keys.size()); + assert(columns.size() >= keys.size()); + Element document; + if (!parser.parse(json.to_string_view(), document) || !document.isObject()) { + return false; + } + const auto& obj = document.getObject(); + for (size_t x = 0; x < types.size(); ++x) { + Element element; + PathInData key_path(keys[x].to_string_view()); + if (!obj.find(key_path, element) || element.isNull()) { + columns[x]->insert_default(); + continue; + } + switch (types[x]) { + case ExtractType::ToString: { + if (element.isString()) { + auto str = element.getString(); + columns[x]->insert_data(str.data(), str.size()); + break; + } + auto str = castValueAsString(element); + columns[x]->insert_data(str.data(), str.size()); + break; + } + default: + break; + } + } + return true; +} + +template +std::optional JSONDataParser::parse(const char* begin, size_t length) { + std::string_view json {begin, length}; + Element document; + if (!parser.parse(json, document)) { + return {}; + } + ParseContext context; + traverse(document, context); + ParseResult result; + result.values = std::move(context.values); + result.paths.reserve(context.paths.size()); + for (auto&& path : context.paths) { + result.paths.emplace_back(std::move(path)); + } + return result; +} + +template +void JSONDataParser::traverse(const Element& element, ParseContext& ctx) { + // checkStackSize(); + if (element.isObject()) { + traverseObject(element.getObject(), ctx); + } else if (element.isArray()) { + traverseArray(element.getArray(), ctx); + } else { + ctx.paths.push_back(ctx.builder.get_parts()); + ctx.values.push_back(getValueAsField(element)); + } +} +template +void JSONDataParser::traverseObject(const JSONObject& object, ParseContext& ctx) { + ctx.paths.reserve(ctx.paths.size() + object.size()); + ctx.values.reserve(ctx.values.size() + object.size()); + for (auto it = object.begin(); it != object.end(); ++it) { + const auto& [key, value] = *it; + ctx.builder.append(key, false); + traverse(value, ctx); + ctx.builder.pop_back(); + } +} +template +void JSONDataParser::traverseArray(const JSONArray& array, ParseContext& ctx) { + /// Traverse elements of array and collect an array of fields by each path. + ParseArrayContext array_ctx; + array_ctx.total_size = array.size(); + for (auto it = array.begin(); it != array.end(); ++it) { + traverseArrayElement(*it, array_ctx); + ++array_ctx.current_size; + } + auto&& arrays_by_path = array_ctx.arrays_by_path; + if (arrays_by_path.empty()) { + ctx.paths.push_back(ctx.builder.get_parts()); + ctx.values.push_back(Array()); + } else { + ctx.paths.reserve(ctx.paths.size() + arrays_by_path.size()); + ctx.values.reserve(ctx.values.size() + arrays_by_path.size()); + for (auto it = arrays_by_path.begin(); it != arrays_by_path.end(); ++it) { + auto&& [path, path_array] = it->second; + /// Merge prefix path and path of array element. + ctx.paths.push_back(ctx.builder.append(path, true).get_parts()); + ctx.values.push_back(std::move(path_array)); + ctx.builder.pop_back(path.size()); + } + } +} +template +void JSONDataParser::traverseArrayElement(const Element& element, + ParseArrayContext& ctx) { + ParseContext element_ctx; + traverse(element, element_ctx); + auto& [_, paths, values] = element_ctx; + size_t size = paths.size(); + size_t keys_to_update = ctx.arrays_by_path.size(); + for (size_t i = 0; i < size; ++i) { + if (values[i].is_null()) { + continue; + } + UInt128 hash = PathInData::get_parts_hash(paths[i]); + auto found = ctx.arrays_by_path.find(hash); + if (found != ctx.arrays_by_path.end()) { + auto& path_array = found->second.second; + assert(path_array.size() == ctx.current_size); + /// If current element of array is part of Nested, + /// collect its size or check it if the size of + /// the Nested has been already collected. + auto nested_key = getNameOfNested(paths[i], values[i]); + if (!nested_key.empty()) { + size_t array_size = get(values[i]).size(); + auto& current_nested_sizes = ctx.nested_sizes_by_key[nested_key]; + if (current_nested_sizes.size() == ctx.current_size) { + current_nested_sizes.push_back(array_size); + } else if (array_size != current_nested_sizes.back()) { + LOG(FATAL) << fmt::format("Array sizes mismatched ({} and {})", array_size, + current_nested_sizes.back()); + } + } + path_array.push_back(std::move(values[i])); + --keys_to_update; + } else { + /// We found a new key. Add and empty array with current size. + Array path_array; + path_array.reserve(ctx.total_size); + path_array.resize(ctx.current_size); + auto nested_key = getNameOfNested(paths[i], values[i]); + if (!nested_key.empty()) { + size_t array_size = get(values[i]).size(); + auto& current_nested_sizes = ctx.nested_sizes_by_key[nested_key]; + if (current_nested_sizes.empty()) { + current_nested_sizes.resize(ctx.current_size); + } else { + /// If newly added element is part of the Nested then + /// resize its elements to keep correct sizes of Nested arrays. + for (size_t j = 0; j < ctx.current_size; ++j) { + path_array[j] = Array(current_nested_sizes[j]); + } + } + if (current_nested_sizes.size() == ctx.current_size) { + current_nested_sizes.push_back(array_size); + } else if (array_size != current_nested_sizes.back()) { + LOG(FATAL) << fmt::format("Array sizes mismatched ({} and {})", array_size, + current_nested_sizes.back()); + } + } + path_array.push_back(std::move(values[i])); + auto& elem = ctx.arrays_by_path[hash]; + elem.first = std::move(paths[i]); + elem.second = std::move(path_array); + } + } + /// If some of the keys are missed in current element, + /// add default values for them. + if (keys_to_update) { + fillMissedValuesInArrays(ctx); + } +} + +template +void JSONDataParser::fillMissedValuesInArrays(ParseArrayContext& ctx) { + for (auto it = ctx.arrays_by_path.begin(); it != ctx.arrays_by_path.end(); ++it) { + auto& [path, path_array] = it->second; + assert(path_array.size() == ctx.current_size || path_array.size() == ctx.current_size + 1); + if (path_array.size() == ctx.current_size) { + bool inserted = tryInsertDefaultFromNested(ctx, path, path_array); + if (!inserted) { + path_array.emplace_back(); + } + } + } +} + +template +bool JSONDataParser::tryInsertDefaultFromNested(ParseArrayContext& ctx, + const PathInData::Parts& path, + Array& array) { + /// If there is a collected size of current Nested + /// then insert array of this size as a default value. + if (path.empty() || array.empty()) { + return false; + } + /// Last element is not Null, because otherwise this path wouldn't exist. + auto nested_key = getNameOfNested(path, array.back()); + if (nested_key.empty()) { + return false; + } + auto mapped = ctx.nested_sizes_by_key.find(nested_key); + if (mapped == ctx.nested_sizes_by_key.end()) { + return false; + } + auto& current_nested_sizes = mapped->second; + assert(current_nested_sizes.size() == ctx.current_size || + current_nested_sizes.size() == ctx.current_size + 1); + /// If all keys of Nested were missed then add a zero length. + if (current_nested_sizes.size() == ctx.current_size) { + current_nested_sizes.push_back(0); + } + size_t array_size = current_nested_sizes.back(); + array.push_back(Array(array_size)); + return true; +} + +template +StringRef JSONDataParser::getNameOfNested(const PathInData::Parts& path, + const Field& value) { + if (value.get_type() != Field::Types::Array || path.empty()) { + return {}; + } + /// Find first key that is marked as nested, + /// because we may have tuple of Nested and there could be + /// several arrays with the same prefix, but with independent sizes. + /// Consider we have array element with type `k2 Tuple(k3 Nested(...), k5 Nested(...))` + /// Then subcolumns `k2.k3` and `k2.k5` may have indepented sizes and we should extract + /// `k3` and `k5` keys instead of `k2`. + for (const auto& part : path) { + if (part.is_nested) { + return StringRef(part.key.data(), part.key.size()); + } + } + return {}; +} + +template class JSONDataParser; +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/json/json_parser.h b/be/src/vec/json/json_parser.h new file mode 100644 index 00000000000000..5b4a6ac659f217 --- /dev/null +++ b/be/src/vec/json/json_parser.h @@ -0,0 +1,119 @@ +// 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. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/JSONParsers/SimdJSONParser.h +// and modified by Doris + +#pragma once + +#include +#include +#include +#include + +namespace doris::vectorized { + +template +static Field getValueAsField(const Element& element) { + // bool will convert to type FiledType::UInt64 + if (element.isBool()) { + return element.getBool(); + } + if (element.isInt64()) { + return element.getInt64(); + } + // doris only support signed integers at present + if (element.isUInt64()) { + return element.getInt64(); + } + if (element.isDouble()) { + return element.getDouble(); + } + if (element.isString()) { + return element.getString(); + } + if (element.isNull()) { + return Field(); + } + return Field(); +} + +template +static std::string castValueAsString(const Element& element) { + if (element.isBool()) { + return element.getBool() ? "1" : "0"; + } + if (element.isInt64()) { + return std::to_string(element.getInt64()); + } + if (element.isUInt64()) { + return std::to_string(element.getUInt64()); + } + if (element.isDouble()) { + return std::to_string(element.getDouble()); + } + if (element.isNull()) { + return ""; + } + return ""; +} + +enum class ExtractType { + ToString = 0, + // ... +}; +template +class JSONDataParser { +public: + using Element = typename ParserImpl::Element; + using JSONObject = typename ParserImpl::Object; + using JSONArray = typename ParserImpl::Array; + std::optional parse(const char* begin, size_t length); + + // extract keys's element into columns + bool extract_key(MutableColumns& columns, StringRef json, const std::vector& keys, + const std::vector& types); + +private: + struct ParseContext { + PathInDataBuilder builder; + std::vector paths; + std::vector values; + }; + using PathPartsWithArray = std::pair; + using PathToArray = phmap::flat_hash_map; + using KeyToSizes = phmap::flat_hash_map, StringRefHash>; + struct ParseArrayContext { + size_t current_size = 0; + size_t total_size = 0; + PathToArray arrays_by_path; + KeyToSizes nested_sizes_by_key; + // Arena strings_pool; + }; + void traverse(const Element& element, ParseContext& ctx); + void traverseObject(const JSONObject& object, ParseContext& ctx); + void traverseArray(const JSONArray& array, ParseContext& ctx); + void traverseArrayElement(const Element& element, ParseArrayContext& ctx); + static void fillMissedValuesInArrays(ParseArrayContext& ctx); + static bool tryInsertDefaultFromNested(ParseArrayContext& ctx, const PathInData::Parts& path, + Array& array); + static StringRef getNameOfNested(const PathInData::Parts& path, const Field& value); + + ParserImpl parser; +}; + +} // namespace doris::vectorized diff --git a/be/src/vec/json/parse2column.cpp b/be/src/vec/json/parse2column.cpp new file mode 100644 index 00000000000000..6a6474c8ad9e31 --- /dev/null +++ b/be/src/vec/json/parse2column.cpp @@ -0,0 +1,285 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace doris::vectorized { + +/** Pool for objects that cannot be used from different threads simultaneously. + * Allows to create an object for each thread. + * Pool has unbounded size and objects are not destroyed before destruction of pool. + * + * Use it in cases when thread local storage is not appropriate + * (when maximum number of simultaneously used objects is less + * than number of running/sleeping threads, that has ever used object, + * and creation/destruction of objects is expensive). + */ +template +class SimpleObjectPool { +protected: + /// Hold all available objects in stack. + std::mutex mutex; + std::stack> stack; + /// Specialized deleter for std::unique_ptr. + /// Returns underlying pointer back to stack thus reclaiming its ownership. + struct Deleter { + SimpleObjectPool* parent; + Deleter(SimpleObjectPool* parent_ = nullptr) : parent {parent_} {} /// NOLINT + void operator()(T* owning_ptr) const { + std::lock_guard lock {parent->mutex}; + parent->stack.emplace(owning_ptr); + } + }; + +public: + using Pointer = std::unique_ptr; + /// Extracts and returns a pointer from the stack if it's not empty, + /// creates a new one by calling provided f() otherwise. + template + Pointer get(Factory&& f) { + std::unique_lock lock(mutex); + if (stack.empty()) { + lock.unlock(); + return {f(), this}; + } + auto object = stack.top().release(); + stack.pop(); + return std::unique_ptr(object, Deleter(this)); + } + /// Like get(), but creates object using default constructor. + Pointer getDefault() { + return get([] { return new T; }); + } +}; + +SimpleObjectPool> parsers_pool; + +using Node = typename ColumnObject::Subcolumns::Node; +/// Visitor that keeps @num_dimensions_to_keep dimensions in arrays +/// and replaces all scalars or nested arrays to @replacement at that level. +class FieldVisitorReplaceScalars : public StaticVisitor { +public: + FieldVisitorReplaceScalars(const Field& replacement_, size_t num_dimensions_to_keep_) + : replacement(replacement_), num_dimensions_to_keep(num_dimensions_to_keep_) {} + template + Field operator()(const T& x) const { + if constexpr (std::is_same_v) { + if (num_dimensions_to_keep == 0) { + return replacement; + } + const size_t size = x.size(); + Array res(size); + for (size_t i = 0; i < size; ++i) { + res[i] = apply_visitor( + FieldVisitorReplaceScalars(replacement, num_dimensions_to_keep - 1), x[i]); + } + return res; + } else { + return replacement; + } + } + +private: + const Field& replacement; + size_t num_dimensions_to_keep; +}; + +/// Finds a subcolumn from the same Nested type as @entry and inserts +/// an array with default values with consistent sizes as in Nested type. +bool try_insert_default_from_nested(const std::shared_ptr& entry, + const ColumnObject::Subcolumns& subcolumns) { + if (!entry->path.has_nested_part()) { + return false; + } + + const Node* current_node = subcolumns.find_leaf(entry->path); + const Node* leaf = nullptr; + size_t num_skipped_nested = 0; + + while (current_node) { + /// Try to find the first Nested up to the current node. + const auto* node_nested = ColumnObject::Subcolumns::find_parent( + current_node, [](const auto& candidate) { return candidate.is_nested(); }); + + if (!node_nested) { + break; + } + + /// If there are no leaves, skip current node and find + /// the next node up to the current. + leaf = ColumnObject::Subcolumns::find_leaf(node_nested, [&](const auto& candidate) { + return candidate.data.size() == entry->data.size() + 1; + }); + + if (leaf) { + break; + } + + current_node = node_nested->parent; + ++num_skipped_nested; + } + + if (!leaf) { + return false; + } + + auto last_field = leaf->data.get_last_field(); + if (last_field.is_null()) { + return false; + } + + const auto& least_common_type = entry->data.get_least_common_type(); + size_t num_dimensions = schema_util::get_number_of_dimensions(*least_common_type); + assert(num_skipped_nested < num_dimensions); + + /// Replace scalars to default values with consistent array sizes. + size_t num_dimensions_to_keep = num_dimensions - num_skipped_nested; + auto default_scalar = + num_skipped_nested + ? schema_util::create_empty_array_field(num_skipped_nested) + : schema_util::get_base_type_of_array(least_common_type)->get_default(); + + auto default_field = apply_visitor( + FieldVisitorReplaceScalars(default_scalar, num_dimensions_to_keep), last_field); + entry->data.insert(std::move(default_field)); + + return true; +} + +template +Status parse_json_to_variant(IColumn& column, const char* src, size_t length, + JSONDataParser* parser) { + auto& column_object = assert_cast(column); + std::optional result; + /// Treat empty string as an empty object + /// for better CAST from String to Object. + if (length > 0) { + result = parser->parse(src, length); + } else { + result = ParseResult {}; + } + if (!result) { + LOG(INFO) << "failed to parse " << std::string_view(src, length) << ", length= " << length; + return Status::DataQualityError( + fmt::format("Failed to parse object {}", std::string_view(src, length))); + } + auto& [paths, values] = *result; + assert(paths.size() == values.size()); + phmap::flat_hash_set paths_set; + size_t num_rows = column_object.size(); + for (size_t i = 0; i < paths.size(); ++i) { + FieldInfo field_info; + RETURN_IF_ERROR(get_field_info(values[i], &field_info)); + // TODO support multi dimensions array + if (!config::enable_parse_multi_dimession_array && field_info.num_dimensions >= 2) { + return Status::DataQualityError( + "Sorry multi dimensions array is not supported now, we are working on it"); + } + if (is_nothing(field_info.scalar_type)) { + continue; + } + if (!paths_set.insert(paths[i].get_path()).second) { + return Status::DataQualityError( + fmt::format("Object has ambiguous path {}, {}", paths[i].get_path())); + } + + if (!column_object.has_subcolumn(paths[i])) { + if (paths[i].has_nested_part()) { + column_object.add_nested_subcolumn(paths[i], field_info, num_rows); + } else { + column_object.add_sub_column(paths[i], num_rows); + } + } + auto* subcolumn = column_object.get_subcolumn(paths[i]); + if (!subcolumn) { + return Status::DataQualityError( + fmt::format("Failed to find sub column {}", paths[i].get_path())); + } + assert(subcolumn->size() == num_rows); + Status st = subcolumn->insert(std::move(values[i]), std::move(field_info)); + if (st.is_invalid_argument()) { + return Status::DataQualityError( + fmt::format("Failed to insert field {}", st.to_string())); + } + RETURN_IF_ERROR(st); + } + // /// Insert default values to missed subcolumns. + const auto& subcolumns = column_object.get_subcolumns(); + for (const auto& entry : subcolumns) { + if (!paths_set.contains(entry->path.get_path())) { + bool inserted = try_insert_default_from_nested(entry, subcolumns); + if (!inserted) { + entry->data.insertDefault(); + } + } + } + column_object.incr_num_rows(); + return Status::OK(); +} + +bool extract_key(MutableColumns& columns, StringRef json, const std::vector& keys, + const std::vector& types, JSONDataParser* parser) { + return parser->extract_key(columns, json, keys, types); +} + +// exposed interfaces +Status parse_json_to_variant(IColumn& column, const StringRef& json, + JSONDataParser* parser) { + return parse_json_to_variant(column, json.data, json.size, parser); +} + +Status parse_json_to_variant(IColumn& column, const std::vector& jsons) { + auto parser = parsers_pool.get([] { return new JSONDataParser(); }); + for (StringRef str : jsons) { + RETURN_IF_ERROR(parse_json_to_variant(column, str.data, str.size, parser.get())); + } + return Status::OK(); +} + +bool extract_key(MutableColumns& columns, const std::vector& jsons, + const std::vector& keys, const std::vector& types) { + auto parser = parsers_pool.get([] { return new JSONDataParser(); }); + for (StringRef json : jsons) { + if (!extract_key(columns, json, keys, types, parser.get())) { + return false; + } + } + return true; +} + +bool extract_key(MutableColumns& columns, const ColumnString& json_column, + const std::vector& keys, const std::vector& types) { + auto parser = parsers_pool.get([] { return new JSONDataParser(); }); + for (size_t x = 0; x < json_column.size(); ++x) { + if (!extract_key(columns, json_column.get_data_at(x), keys, types, parser.get())) { + return false; + } + } + return true; +} + +} // namespace doris::vectorized diff --git a/be/src/vec/json/parse2column.h b/be/src/vec/json/parse2column.h new file mode 100644 index 00000000000000..adf1c63e622f39 --- /dev/null +++ b/be/src/vec/json/parse2column.h @@ -0,0 +1,40 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +#pragma once + +#include +#include +#include +#include + +namespace doris::vectorized { + +// parse a batch of json strings into column object +Status parse_json_to_variant(IColumn& column, const std::vector& jsons); + +// parse a single json +Status parse_json_to_variant(IColumn& column, const StringRef& jsons, + JSONDataParser* parser); + +// extract keys columns from json strings into columns +bool extract_key(MutableColumns& columns, const std::vector& jsons, + const std::vector& keys, const std::vector& types); + +// extract keys columns from colunnstring(json format) into columns +bool extract_key(MutableColumns& columns, const ColumnString& json_column, + const std::vector& keys, const std::vector& types); +} // namespace doris::vectorized diff --git a/be/src/vec/json/path_in_data.cpp b/be/src/vec/json/path_in_data.cpp new file mode 100644 index 00000000000000..0ee4ce85287df1 --- /dev/null +++ b/be/src/vec/json/path_in_data.cpp @@ -0,0 +1,134 @@ +// 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. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/DataTypes/Serializations/PathInData.cpp +// and modified by Doris + +#include +#include + +#include +#include +namespace doris::vectorized { +PathInData::PathInData(std::string_view path_) : path(path_) { + const char* begin = path.data(); + const char* end = path.data() + path.size(); + for (const char* it = path.data(); it != end; ++it) { + if (*it == '.') { + size_t size = static_cast(it - begin); + parts.emplace_back(std::string_view {begin, size}, false, 0); + begin = it + 1; + } + } + size_t size = static_cast(end - begin); + parts.emplace_back(std::string_view {begin, size}, false, 0.); +} +PathInData::PathInData(const Parts& parts_) { + build_path(parts_); + build_parts(parts_); +} +PathInData::PathInData(const PathInData& other) : path(other.path) { + build_parts(other.get_parts()); +} +PathInData& PathInData::operator=(const PathInData& other) { + if (this != &other) { + path = other.path; + build_parts(other.parts); + } + return *this; +} +UInt128 PathInData::get_parts_hash(const Parts& parts_) { + SipHash hash; + hash.update(parts_.size()); + for (const auto& part : parts_) { + hash.update(part.key.data(), part.key.length()); + hash.update(part.is_nested); + hash.update(part.anonymous_array_level); + } + UInt128 res; + hash.get128(res); + return res; +} + +void PathInData::build_path(const Parts& other_parts) { + if (other_parts.empty()) { + return; + } + path.clear(); + auto it = other_parts.begin(); + path += it->key; + ++it; + for (; it != other_parts.end(); ++it) { + path += "."; + path += it->key; + } +} +void PathInData::build_parts(const Parts& other_parts) { + if (other_parts.empty()) { + return; + } + parts.clear(); + parts.reserve(other_parts.size()); + const char* begin = path.data(); + for (const auto& part : other_parts) { + has_nested |= part.is_nested; + parts.emplace_back(std::string_view {begin, part.key.length()}, part.is_nested, + part.anonymous_array_level); + begin += part.key.length() + 1; + } +} +size_t PathInData::Hash::operator()(const PathInData& value) const { + auto hash = get_parts_hash(value.parts); + return hash.low ^ hash.high; +} +PathInDataBuilder& PathInDataBuilder::append(std::string_view key, bool is_array) { + if (parts.empty()) { + current_anonymous_array_level += is_array; + } + if (!key.empty()) { + if (!parts.empty()) { + parts.back().is_nested = is_array; + } + parts.emplace_back(key, false, current_anonymous_array_level); + current_anonymous_array_level = 0; + } + return *this; +} +PathInDataBuilder& PathInDataBuilder::append(const PathInData::Parts& path, bool is_array) { + if (parts.empty()) { + current_anonymous_array_level += is_array; + } + if (!path.empty()) { + if (!parts.empty()) { + parts.back().is_nested = is_array; + } + auto it = parts.insert(parts.end(), path.begin(), path.end()); + for (; it != parts.end(); ++it) { + it->anonymous_array_level += current_anonymous_array_level; + } + current_anonymous_array_level = 0; + } + return *this; +} +void PathInDataBuilder::pop_back() { + parts.pop_back(); +} +void PathInDataBuilder::pop_back(size_t n) { + assert(n <= parts.size()); + parts.resize(parts.size() - n); +} +} // namespace doris::vectorized diff --git a/be/src/vec/json/path_in_data.h b/be/src/vec/json/path_in_data.h new file mode 100644 index 00000000000000..5118409e53c187 --- /dev/null +++ b/be/src/vec/json/path_in_data.h @@ -0,0 +1,107 @@ +// 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. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/DataTypes/Serializations/PathInData.h +// and modified by Doris + +#pragma once + +#include +#include + +#include "vec/core/field.h" +#include "vec/core/types.h" +namespace doris::vectorized { +class ReadBuffer; +class WriteBuffer; +/// Class that represents path in document, e.g. JSON. +class PathInData { +public: + struct Part { + Part() = default; + Part(std::string_view key_, bool is_nested_, vectorized::UInt8 anonymous_array_level_) + : key(key_), is_nested(is_nested_), anonymous_array_level(anonymous_array_level_) {} + bool operator==(const Part& other) const { + return this->key == other.key && this->is_nested == other.is_nested && + this->anonymous_array_level == other.anonymous_array_level; + } + /// Name of part of path. + std::string_view key; + /// If this part is Nested, i.e. element + /// related to this key is the array of objects. + bool is_nested = false; + /// Number of array levels between current key and previous key. + /// E.g. in JSON {"k1": [[[{"k2": 1, "k3": 2}]]]} + /// "k1" is nested and has anonymous_array_level = 0. + /// "k2" and "k3" are not nested and have anonymous_array_level = 2. + UInt8 anonymous_array_level = 0; + }; + using Parts = std::vector; + PathInData() = default; + explicit PathInData(std::string_view path_); + explicit PathInData(const Parts& parts_); + PathInData(const PathInData& other); + PathInData& operator=(const PathInData& other); + static UInt128 get_parts_hash(const Parts& parts_); + bool empty() const { return parts.empty(); } + const vectorized::String& get_path() const { return path; } + const Parts& get_parts() const { return parts; } + bool is_nested(size_t i) const { return parts[i].is_nested; } + bool has_nested_part() const { return has_nested; } + bool operator==(const PathInData& other) const { return parts == other.parts; } + struct Hash { + size_t operator()(const PathInData& value) const; + }; + +private: + /// Creates full path from parts. + void build_path(const Parts& other_parts); + /// Creates new parts full from full path with correct string pointers. + void build_parts(const Parts& other_parts); + /// The full path. Parts are separated by dots. + vectorized::String path; + /// Parts of the path. All string_view-s in parts must point to the @path. + Parts parts; + /// True if at least one part is nested. + /// Cached to avoid linear complexity at 'has_nested'. + bool has_nested = false; +}; +class PathInDataBuilder { +public: + const PathInData::Parts& get_parts() const { return parts; } + PathInDataBuilder& append(std::string_view key, bool is_array); + PathInDataBuilder& append(const PathInData::Parts& path, bool is_array); + void pop_back(); + void pop_back(size_t n); + +private: + PathInData::Parts parts; + /// Number of array levels without key to which + /// next non-empty key will be nested. + /// Example: for JSON { "k1": [[{"k2": 1, "k3": 2}] } + // `k2` and `k3` has anonymous_array_level = 1 in that case. + size_t current_anonymous_array_level = 0; +}; +using PathsInData = std::vector; +/// Result of parsing of a document. +/// Contains all paths extracted from document +/// and values which are related to them. +struct ParseResult { + std::vector paths; + std::vector values; +}; +} // namespace doris::vectorized diff --git a/be/src/vec/json/simd_json_parser.h b/be/src/vec/json/simd_json_parser.h new file mode 100644 index 00000000000000..870ae6b9474405 --- /dev/null +++ b/be/src/vec/json/simd_json_parser.h @@ -0,0 +1,225 @@ +// 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. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/JSONParsers/SimdJSONParser.h +// and modified by Doris + +#pragma once + +#include +#include +#include +#include + +namespace doris::vectorized { + +/// This class can be used as an argument for the template class FunctionJSON. +/// It provides ability to parse JSONs using simdjson library. +class SimdJSONParser { +public: + class Array; + class Object; + /// References an element in a JSON document, representing a JSON null, boolean, string, number, + /// array or object. + class Element { + public: + ALWAYS_INLINE Element() {} /// NOLINT + ALWAYS_INLINE Element(const simdjson::dom::element& element_) + : element(element_) {} /// NOLINT + ALWAYS_INLINE bool isInt64() const { + return element.type() == simdjson::dom::element_type::INT64; + } + ALWAYS_INLINE bool isUInt64() const { + return element.type() == simdjson::dom::element_type::UINT64; + } + ALWAYS_INLINE bool isDouble() const { + return element.type() == simdjson::dom::element_type::DOUBLE; + } + ALWAYS_INLINE bool isString() const { + return element.type() == simdjson::dom::element_type::STRING; + } + ALWAYS_INLINE bool isArray() const { + return element.type() == simdjson::dom::element_type::ARRAY; + } + ALWAYS_INLINE bool isObject() const { + return element.type() == simdjson::dom::element_type::OBJECT; + } + ALWAYS_INLINE bool isBool() const { + return element.type() == simdjson::dom::element_type::BOOLEAN; + } + ALWAYS_INLINE bool isNull() const { + return element.type() == simdjson::dom::element_type::NULL_VALUE; + } + ALWAYS_INLINE Int64 getInt64() const { return element.get_int64().value_unsafe(); } + ALWAYS_INLINE UInt64 getUInt64() const { return element.get_uint64().value_unsafe(); } + ALWAYS_INLINE double getDouble() const { return element.get_double().value_unsafe(); } + ALWAYS_INLINE bool getBool() const { return element.get_bool().value_unsafe(); } + ALWAYS_INLINE std::string_view getString() const { + return element.get_string().value_unsafe(); + } + ALWAYS_INLINE Array getArray() const; + ALWAYS_INLINE Object getObject() const; + ALWAYS_INLINE simdjson::dom::element getElement() const { return element; } + + private: + simdjson::dom::element element; + }; + /// References an array in a JSON document. + class Array { + public: + class Iterator { + public: + ALWAYS_INLINE Iterator(const simdjson::dom::array::iterator& it_) + : it(it_) {} /// NOLINT + ALWAYS_INLINE Element operator*() const { return *it; } + ALWAYS_INLINE Iterator& operator++() { + ++it; + return *this; + } + ALWAYS_INLINE Iterator operator++(int) { + auto res = *this; + ++it; + return res; + } /// NOLINT + ALWAYS_INLINE friend bool operator!=(const Iterator& left, const Iterator& right) { + return left.it != right.it; + } + ALWAYS_INLINE friend bool operator==(const Iterator& left, const Iterator& right) { + return !(left != right); + } + + private: + simdjson::dom::array::iterator it; + }; + ALWAYS_INLINE Array(const simdjson::dom::array& array_) : array(array_) {} /// NOLINT + ALWAYS_INLINE Iterator begin() const { return array.begin(); } + ALWAYS_INLINE Iterator end() const { return array.end(); } + ALWAYS_INLINE size_t size() const { return array.size(); } + ALWAYS_INLINE Element operator[](size_t index) const { + assert(index < size()); + return array.at(index).value_unsafe(); + } + + private: + simdjson::dom::array array; + }; + using KeyValuePair = std::pair; + /// References an object in a JSON document. + class Object { + public: + class Iterator { + public: + ALWAYS_INLINE Iterator(const simdjson::dom::object::iterator& it_) + : it(it_) {} /// NOLINT + ALWAYS_INLINE KeyValuePair operator*() const { + const auto& res = *it; + return {res.key, res.value}; + } + ALWAYS_INLINE Iterator& operator++() { + ++it; + return *this; + } + ALWAYS_INLINE Iterator operator++(int) { + auto res = *this; + ++it; + return res; + } /// NOLINT + ALWAYS_INLINE friend bool operator!=(const Iterator& left, const Iterator& right) { + return left.it != right.it; + } + ALWAYS_INLINE friend bool operator==(const Iterator& left, const Iterator& right) { + return !(left != right); + } + + private: + simdjson::dom::object::iterator it; + }; + ALWAYS_INLINE Object(const simdjson::dom::object& object_) : object(object_) {} /// NOLINT + ALWAYS_INLINE Iterator begin() const { return object.begin(); } + ALWAYS_INLINE Iterator end() const { return object.end(); } + ALWAYS_INLINE size_t size() const { return object.size(); } + bool find(const PathInData& path, Element& result) const { + size_t idx = 0; + auto obj = object; + simdjson::simdjson_result x; + for (const auto& part : path.get_parts()) { + ++idx; + x = obj.at_key(part.key); + if (x.error()) { + return false; + } + if (!x.is_object()) { + break; + } + obj = x.get_object(); + } + if (idx != path.get_parts().size()) { + return false; + } + result = x.value_unsafe(); + return true; + } + bool find(const std::string_view& key, Element& result) const { + auto x = object.at_key(key); + if (x.error()) { + return false; + } + result = x.value_unsafe(); + return true; + } + /// Optional: Provides access to an object's element by index. + KeyValuePair operator[](size_t index) const { + assert(index < size()); + auto it = object.begin(); + while (index--) { + ++it; + } + const auto& res = *it; + return {res.key, res.value}; + } + + private: + simdjson::dom::object object; + }; + /// Parses a JSON document, returns the reference to its root element if succeeded. + bool parse(const std::string_view& json, Element& result) { + auto document = parser.parse(json.data(), json.size()); + if (document.error()) { + return false; + } + result = document.value_unsafe(); + return true; + } + /// Optional: Allocates memory to parse JSON documents faster. + void reserve(size_t max_size) { + if (parser.allocate(max_size) != simdjson::error_code::SUCCESS) { + LOG(FATAL) << "Couldn't allocate " + std::to_string(max_size) + + " bytes when parsing JSON"; + } + } + +private: + simdjson::dom::parser parser; +}; +inline ALWAYS_INLINE SimdJSONParser::Array SimdJSONParser::Element::getArray() const { + return element.get_array().value_unsafe(); +} +inline ALWAYS_INLINE SimdJSONParser::Object SimdJSONParser::Element::getObject() const { + return element.get_object().value_unsafe(); +} + +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/olap/olap_data_convertor.h b/be/src/vec/olap/olap_data_convertor.h index 1b736966979e5a..d6c65cc294f553 100644 --- a/be/src/vec/olap/olap_data_convertor.h +++ b/be/src/vec/olap/olap_data_convertor.h @@ -56,6 +56,7 @@ class OlapBlockDataConvertor { std::pair convert_column_data(size_t cid); void add_column_data_convertor(const TabletColumn& column); + bool empty() const { return _convertors.empty(); } void reserve(size_t size) { _convertors.reserve(size); } private: diff --git a/be/src/vec/sink/vtablet_sink.cpp b/be/src/vec/sink/vtablet_sink.cpp index 0ad8458f13a26c..54f761955b2c87 100644 --- a/be/src/vec/sink/vtablet_sink.cpp +++ b/be/src/vec/sink/vtablet_sink.cpp @@ -225,8 +225,6 @@ Status VNodeChannel::init(RuntimeState* state) { _rpc_timeout_ms = state->query_options().query_timeout * 1000; _timeout_watch.start(); - _cur_mutable_block.reset(new vectorized::MutableBlock({_tuple_desc})); - // Initialize _cur_add_block_request _cur_add_block_request.set_allocated_id(&_parent->_load_id); _cur_add_block_request.set_index_id(_index_channel->_index_id); @@ -428,7 +426,16 @@ Status VNodeChannel::add_block(vectorized::Block* block, std::this_thread::sleep_for(std::chrono::milliseconds(10)); } - block->append_block_by_selector(_cur_mutable_block->mutable_columns(), *(payload.first)); + if (UNLIKELY(!_cur_mutable_block)) { + _cur_mutable_block.reset(new vectorized::MutableBlock(block->clone_empty())); + } + + if (_parent->_schema->is_dynamic_schema()) { + // Set _cur_mutable_block to dynamic since input blocks may be structure-variable(dyanmic) + // this will align _cur_mutable_block with block and auto extends columns + _cur_mutable_block->set_block_type(vectorized::BlockType::DYNAMIC); + } + block->append_block_by_selector(_cur_mutable_block.get(), *(payload.first)); for (auto tablet_id : payload.second) { _cur_add_block_request.add_tablet_ids(tablet_id); } @@ -447,8 +454,7 @@ Status VNodeChannel::add_block(vectorized::Block* block, << " jobid:" << std::to_string(_state->load_job_id()) << " loadinfo:" << _load_info; } - - _cur_mutable_block.reset(new vectorized::MutableBlock({_tuple_desc})); + _cur_mutable_block.reset(new vectorized::MutableBlock(block->clone_empty())); _cur_add_block_request.clear_tablet_ids(); } @@ -731,6 +737,10 @@ void VNodeChannel::mark_close() { { debug::ScopedTSANIgnoreReadsAndWrites ignore_tsan; std::lock_guard l(_pending_batches_lock); + if (!_cur_mutable_block) { + // add a dummy block + _cur_mutable_block.reset(new vectorized::MutableBlock()); + } _pending_blocks.emplace(std::move(_cur_mutable_block), _cur_add_block_request); _pending_batches_num++; DCHECK(_pending_blocks.back().second.eos()); @@ -1453,7 +1463,7 @@ Status VOlapTableSink::_validate_data(RuntimeState* state, vectorized::Block* bl } void VOlapTableSink::_convert_to_dest_desc_block(doris::vectorized::Block* block) { - for (int i = 0; i < _output_tuple_desc->slots().size(); ++i) { + for (int i = 0; i < _output_tuple_desc->slots().size() && i < block->columns(); ++i) { SlotDescriptor* desc = _output_tuple_desc->slots()[i]; if (desc->is_nullable() != block->get_by_position(i).type->is_nullable()) { if (desc->is_nullable()) { diff --git a/be/test/olap/test_data/header_without_inc_rs.txt b/be/test/olap/test_data/header_without_inc_rs.txt index 1aa0121ee86b6f..e44ef03355e03d 100644 --- a/be/test/olap/test_data/header_without_inc_rs.txt +++ b/be/test/olap/test_data/header_without_inc_rs.txt @@ -57,7 +57,8 @@ "compression_type": "LZ4F", "schema_version": 0, "disable_auto_compaction": false, - "store_row_column": false + "store_row_column": false, + "is_dynamic_schema": false }, "rs_metas": [ { diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/PrimitiveType.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/PrimitiveType.java index b14d55010252ca..b4449ae560b34d 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/catalog/PrimitiveType.java +++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/PrimitiveType.java @@ -71,6 +71,7 @@ public enum PrimitiveType { // Aligning to 16 bytes total. STRUCT("STRUCT", 16, TPrimitiveType.STRUCT), STRING("STRING", 16, TPrimitiveType.STRING), + VARIANT("VARIANT", 24, TPrimitiveType.VARIANT), // Unsupported scalar types. BINARY("BINARY", -1, TPrimitiveType.BINARY), ALL("ALL", -1, TPrimitiveType.INVALID_TYPE); @@ -557,6 +558,7 @@ public static ImmutableSetMultimap getImplicitCast supportedTypes.add(ARRAY); supportedTypes.add(MAP); supportedTypes.add(QUANTILE_STATE); + supportedTypes.add(VARIANT); } public static ArrayList getIntegerTypes() { @@ -1004,6 +1006,8 @@ public static PrimitiveType fromThrift(TPrimitiveType tPrimitiveType) { return STRUCT; case ALL: return ALL; + case VARIANT: + return VARIANT; default: return INVALID_TYPE; } @@ -1112,6 +1116,10 @@ public boolean isComplexType() { return this == HLL || this == BITMAP; } + public boolean isVariantType() { + return this == VARIANT; + } + public boolean isStringType() { return (this == VARCHAR || this == CHAR || this == HLL || this == STRING); } diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java index fe1fffbc10ecbc..342a5e3b004fb9 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java +++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java @@ -603,6 +603,7 @@ public String toSql(int depth) { case DATEV2: case HLL: case BITMAP: + case VARIANT: case QUANTILE_STATE: stringBuilder.append(type.toString().toLowerCase()); break; diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/Type.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/Type.java index 9e98ed01e59b30..56fbb20a1e9ded 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/catalog/Type.java +++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/Type.java @@ -106,6 +106,7 @@ public abstract class Type { public static final MapType MAP = new MapType(); public static final ArrayType ARRAY = ArrayType.create(); public static final StructType STRUCT = new StructType(); + public static final VariantType VARIANT = new VariantType(); private static final Logger LOG = LogManager.getLogger(Type.class); private static final ArrayList integerTypes; @@ -474,6 +475,10 @@ public boolean isComplexType() { return isStructType() || isCollectionType(); } + public boolean isVariantType() { + return this instanceof VariantType; + } + public boolean isCollectionType() { return isMapType() || isArrayType() || isMultiRowType() || isStructType(); } @@ -780,6 +785,8 @@ public static Type fromPrimitiveType(PrimitiveType type) { return Type.BITMAP; case QUANTILE_STATE: return Type.QUANTILE_STATE; + case VARIANT: + return new VariantType(); default: return null; } @@ -1553,7 +1560,8 @@ public Integer getNumPrecRadix() { || t1 == PrimitiveType.TIMEV2 || t2 == PrimitiveType.TIMEV2 || t1 == PrimitiveType.MAP || t2 == PrimitiveType.MAP || t1 == PrimitiveType.STRUCT || t2 == PrimitiveType.STRUCT - || t1 == PrimitiveType.UNSUPPORTED || t2 == PrimitiveType.UNSUPPORTED) { + || t1 == PrimitiveType.UNSUPPORTED || t2 == PrimitiveType.UNSUPPORTED + || t1 == PrimitiveType.VARIANT || t2 == PrimitiveType.VARIANT) { continue; } Preconditions.checkNotNull(compatibilityMatrix[i][j]); diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/VariantType.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/VariantType.java new file mode 100644 index 00000000000000..e9363dd0141914 --- /dev/null +++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/VariantType.java @@ -0,0 +1,80 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.catalog; + +import org.apache.doris.thrift.TTypeDesc; +import org.apache.doris.thrift.TTypeNode; +import org.apache.doris.thrift.TTypeNodeType; + +public class VariantType extends Type { + public VariantType() { + + } + + @Override + public String toSql(int depth) { + return "VARIANT"; + } + + @Override + protected String prettyPrint(int lpad) { + return "VARIANT"; + } + + @Override + public boolean equals(Object other) { + return other instanceof VariantType; + } + + @Override + public void toThrift(TTypeDesc container) { + TTypeNode node = new TTypeNode(); + container.types.add(node); + node.setType(TTypeNodeType.VARIANT); + } + + @Override + public String toString() { + return toSql(0); + } + + @Override + public PrimitiveType getPrimitiveType() { + return PrimitiveType.VARIANT; + } + + @Override + public boolean supportsTablePartitioning() { + return false; + } + + @Override + public int getSlotSize() { + return PrimitiveType.VARIANT.getSlotSize(); + } + + @Override + public boolean isSupported() { + return true; + } + + @Override + public boolean matchesType(Type t) { + return t.isVariantType() || t.isStringType(); + } +} diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index e3383d04afaf51..97784ab613d555 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -1811,6 +1811,19 @@ create_stmt ::= RESULT = new CreateTableStmt(ifNotExists, isExternal, name, columns, engineName, keys, partition, distribution, tblProperties, extProperties, tableComment, index); :} + | KW_CREATE opt_external:isExternal KW_TABLE opt_if_not_exists:ifNotExists table_name:name + LPAREN column_definition_list:columns COMMA DOTDOTDOT RPAREN opt_engine:engineName + opt_keys:keys + opt_comment:tableComment + opt_partition:partition + opt_distribution:distribution + opt_rollup:index + opt_properties:tblProperties + opt_ext_properties:extProperties + {: + RESULT = new CreateTableStmt(ifNotExists, isExternal, name, columns, null, engineName, keys, partition, + distribution, tblProperties, extProperties, tableComment, index, true); + :} | KW_CREATE opt_external:isExternal KW_TABLE opt_if_not_exists:ifNotExists table_name:name LPAREN column_definition_list:columns COMMA index_definition_list:indexes RPAREN opt_engine:engineName opt_keys:keys @@ -1822,7 +1835,20 @@ create_stmt ::= opt_ext_properties:extProperties {: RESULT = new CreateTableStmt(ifNotExists, isExternal, name, columns, indexes, engineName, keys, partition, - distribution, tblProperties, extProperties, tableComment, index); + distribution, tblProperties, extProperties, tableComment, index, false); + :} + | KW_CREATE opt_external:isExternal KW_TABLE opt_if_not_exists:ifNotExists table_name:name + LPAREN column_definition_list:columns COMMA index_definition_list:indexes COMMA DOTDOTDOT RPAREN opt_engine:engineName + opt_keys:keys + opt_comment:tableComment + opt_partition:partition + opt_distribution:distribution + opt_rollup:index + opt_properties:tblProperties + opt_ext_properties:extProperties + {: + RESULT = new CreateTableStmt(ifNotExists, isExternal, name, columns, indexes, engineName, keys, partition, + distribution, tblProperties, extProperties, tableComment, index, true); :} | KW_CREATE opt_external:isExternal KW_TABLE opt_if_not_exists:ifNotExists table_name:name KW_ENGINE EQUAL ident:engineName properties:properties opt_comment:tableComment diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java index b47052c6010ad7..f95425e2f6d1fc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java @@ -249,7 +249,8 @@ protected void runPendingJob() throws AlterCancelException { tbl.getCompressionType(), tbl.getEnableUniqueKeyMergeOnWrite(), tbl.getStoragePolicy(), tbl.disableAutoCompaction(), - tbl.storeRowColumn()); + tbl.storeRowColumn(), + tbl.isDynamicSchema()); createReplicaTask.setBaseTablet(tabletIdMap.get(rollupTabletId), baseSchemaHash); if (this.storageFormat != null) { createReplicaTask.setStorageFormat(this.storageFormat); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java index 6fc5eec06eae90..2af14cea2cb26d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java @@ -296,7 +296,8 @@ protected void runPendingJob() throws AlterCancelException { tbl.getCompressionType(), tbl.getEnableUniqueKeyMergeOnWrite(), tbl.getStoragePolicy(), tbl.disableAutoCompaction(), - tbl.storeRowColumn()); + tbl.storeRowColumn(), + tbl.isDynamicSchema()); createReplicaTask.setBaseTablet(partitionIndexTabletMap.get(partitionId, shadowIdxId) .get(shadowTabletId), originSchemaHash); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java index 5d9fd884aaefa2..dc366b6dd38294 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java @@ -98,6 +98,8 @@ public DefaultValue(boolean isSet, String value, String exprName) { public static DefaultValue HLL_EMPTY_DEFAULT_VALUE = new DefaultValue(true, ZERO); // default "value", "0" means empty bitmap public static DefaultValue BITMAP_EMPTY_DEFAULT_VALUE = new DefaultValue(true, ZERO); + // default "value", "[]" means empty array + public static DefaultValue ARRAY_EMPTY_DEFAULT_VALUE = new DefaultValue(true, "[]"); public boolean isCurrentTimeStamp() { return "CURRENT_TIMESTAMP".equals(value) && NOW.equals(defaultValueExprDef.getExprName()); @@ -290,8 +292,10 @@ public void analyze(boolean isOlap) throws AnalysisException { throw new AnalysisException("Array can only be used in the non-key column of" + " the duplicate table at present."); } - if (defaultValue.isSet && defaultValue != DefaultValue.NULL_DEFAULT_VALUE) { - throw new AnalysisException("Array type column default value only support null"); + if (defaultValue.isSet && defaultValue != DefaultValue.NULL_DEFAULT_VALUE + && !defaultValue.value.equals(DefaultValue.ARRAY_EMPTY_DEFAULT_VALUE.value)) { + throw new AnalysisException("Array type column default value only support null or " + + DefaultValue.ARRAY_EMPTY_DEFAULT_VALUE.value); } } if (isKey() && type.getPrimitiveType() == PrimitiveType.STRING && isOlap) { @@ -337,7 +341,7 @@ public void analyze(boolean isOlap) throws AnalysisException { throw new AnalysisException("Can not set null default value to non nullable column: " + name); } - if (defaultValue.isSet && defaultValue.value != null) { + if (type.isScalarType() && defaultValue.isSet && defaultValue.value != null) { validateDefaultValue(type, defaultValue.value, defaultValue.defaultValueExprDef); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java index 5e39b4352a6c8b..49d9a37ee11a07 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java @@ -42,6 +42,7 @@ import com.google.common.base.Strings; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.commons.collections.CollectionUtils; import org.apache.logging.log4j.LogManager; @@ -140,7 +141,7 @@ public CreateTableStmt(boolean ifNotExists, Map extProperties, String comment) { this(ifNotExists, isExternal, tableName, columnDefinitions, null, engineName, keysDesc, partitionDesc, - distributionDesc, properties, extProperties, comment, null); + distributionDesc, properties, extProperties, comment, null, false); } public CreateTableStmt(boolean ifNotExists, @@ -155,7 +156,7 @@ public CreateTableStmt(boolean ifNotExists, Map extProperties, String comment, List ops) { this(ifNotExists, isExternal, tableName, columnDefinitions, null, engineName, keysDesc, partitionDesc, - distributionDesc, properties, extProperties, comment, ops); + distributionDesc, properties, extProperties, comment, ops, false); } public CreateTableStmt(boolean ifNotExists, @@ -169,7 +170,8 @@ public CreateTableStmt(boolean ifNotExists, DistributionDesc distributionDesc, Map properties, Map extProperties, - String comment, List rollupAlterClauseList) { + String comment, List rollupAlterClauseList, + boolean isDynamicSchema) { this.tableName = tableName; if (columnDefinitions == null) { this.columnDefs = Lists.newArrayList(); @@ -186,6 +188,12 @@ public CreateTableStmt(boolean ifNotExists, this.keysDesc = keysDesc; this.partitionDesc = partitionDesc; this.distributionDesc = distributionDesc; + if (isDynamicSchema) { + if (properties == null) { + properties = Maps.newHashMap(); + } + properties.put(PropertyAnalyzer.PROPERTIES_DYNAMIC_SCHEMA, "true"); + } this.properties = properties; this.extProperties = extProperties; this.isExternal = isExternal; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TupleDescriptor.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TupleDescriptor.java index d27c8f47258370..01b099a2ca21c4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/TupleDescriptor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TupleDescriptor.java @@ -70,6 +70,8 @@ public class TupleDescriptor { private float avgSerializedSize; // in bytes; includes serialization overhead + private int tableId = -1; + public TupleDescriptor(TupleId id) { this.id = id; this.slots = new ArrayList(); @@ -105,6 +107,10 @@ public ArrayList getSlots() { return slots; } + public void setTableId(int id) { + tableId = id; + } + /** * get slot desc by slot id. * @@ -206,6 +212,9 @@ public TTupleDescriptor toThrift() { if (table != null && table.getId() >= 0) { ttupleDesc.setTableId((int) table.getId()); } + if (tableId > 0) { + ttupleDesc.setTableId(tableId); + } return ttupleDesc; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java index c89c6592074547..c75259050d833e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TypeDef.java @@ -28,6 +28,10 @@ import org.apache.doris.catalog.StructType; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; +import org.apache.doris.thrift.TColumnDesc; +import org.apache.doris.thrift.TPrimitiveType; + +import com.google.common.base.Preconditions; import java.util.ArrayList; import java.util.HashSet; @@ -68,6 +72,26 @@ public static TypeDef createChar(int len) { return new TypeDef(ScalarType.createChar(len)); } + public static Type createType(TColumnDesc tColumnDesc) { + TPrimitiveType tPrimitiveType = tColumnDesc.getColumnType(); + PrimitiveType ptype = PrimitiveType.fromThrift(tPrimitiveType); + if (ptype.isArrayType()) { + // just support array for now + Preconditions.checkState(tColumnDesc.getChildren().size() == 1); + return new ArrayType(createType(tColumnDesc.getChildren().get(0)), + tColumnDesc.getChildren().get(0).isIsAllowNull()); + } + // scarlar type + int columnLength = tColumnDesc.getColumnLength(); + int columnPrecision = tColumnDesc.getColumnPrecision(); + int columnScale = tColumnDesc.getColumnScale(); + return ScalarType.createType(ptype, columnLength, columnPrecision, columnScale); + } + + public static TypeDef createTypeDef(TColumnDesc tcolumnDef) { + return new TypeDef(createType(tcolumnDef)); + } + @Override public void analyze(Analyzer analyzer) throws AnalysisException { if (isAnalyzed) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java index 8b894fc9202caa..d374487216637a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java @@ -1006,7 +1006,8 @@ private void createReplicas(Database db, AgentBatchTask batchTask, OlapTable loc localTbl.getCompressionType(), localTbl.getEnableUniqueKeyMergeOnWrite(), localTbl.getStoragePolicy(), localTbl.disableAutoCompaction(), - localTbl.storeRowColumn()); + localTbl.storeRowColumn(), + localTbl.isDynamicSchema()); task.setInRestoreMode(true); batchTask.addTask(task); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java index a4f00f4bf3cfda..f363b66c2d67b3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java @@ -60,6 +60,7 @@ public class Column implements Writable, GsonPostProcessable { public static final String SEQUENCE_COL = "__DORIS_SEQUENCE_COL__"; public static final String ROWID_COL = "__DORIS_ROWID_COL__"; public static final String ROW_STORE_COL = "__DORIS_ROW_STORE_COL__"; + public static final String DYNAMIC_COLUMN_NAME = "__DORIS_DYNAMIC_COL__"; private static final String COLUMN_ARRAY_CHILDREN = "item"; private static final String COLUMN_STRUCT_CHILDREN = "field"; public static final int COLUMN_UNIQUE_ID_INIT_VALUE = -1; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index ad98182dfba1ba..332e024df27e8b 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -2918,6 +2918,12 @@ public static void getDdlStmt(DdlStmt ddlStmt, String dbName, TableIf table, Lis sb.append(olapTable.storeRowColumn()).append("\""); } + // dynamic schema + if (olapTable.isDynamicSchema()) { + sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_DYNAMIC_SCHEMA).append("\" = \""); + sb.append(olapTable.isDynamicSchema()).append("\""); + } + // disable auto compaction sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_DISABLE_AUTO_COMPACTION).append("\" = \""); sb.append(olapTable.disableAutoCompaction()).append("\""); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index 13c7ae42180960..b6bfa3edb84498 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -1731,6 +1731,22 @@ public Boolean storeRowColumn() { return false; } + public Boolean isDynamicSchema() { + if (tableProperty != null) { + return tableProperty.isDynamicSchema(); + } + return false; + } + + public void setIsDynamicSchema(boolean isDynamicSchema) { + if (tableProperty == null) { + tableProperty = new TableProperty(new HashMap<>()); + } + tableProperty.modifyTableProperties( + PropertyAnalyzer.PROPERTIES_DYNAMIC_SCHEMA, Boolean.valueOf(isDynamicSchema).toString()); + tableProperty.buildDynamicSchema(); + } + public int getBaseSchemaVersion() { MaterializedIndexMeta baseIndexMeta = indexIdToMeta.get(baseIndexId); return baseIndexMeta.getSchemaVersion(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java index a122e004ff98d8..3eaf136fbc7131 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java @@ -60,6 +60,7 @@ public class TableProperty implements Writable { private boolean isInMemory = false; private String storagePolicy = ""; + private boolean isDynamicSchema = false; /* * the default storage format of this table. @@ -188,6 +189,12 @@ public String getStoragePolicy() { return storagePolicy; } + public TableProperty buildDynamicSchema() { + isDynamicSchema = Boolean.parseBoolean( + properties.getOrDefault(PropertyAnalyzer.PROPERTIES_DYNAMIC_SCHEMA, "false")); + return this; + } + public TableProperty buildDataSortInfo() { HashMap dataSortInfoProperties = new HashMap<>(); for (Map.Entry entry : properties.entrySet()) { @@ -266,6 +273,10 @@ public String getEstimatePartitionSize() { return properties.getOrDefault(PropertyAnalyzer.PROPERTIES_ESTIMATE_PARTITION_SIZE, ""); } + public boolean isDynamicSchema() { + return isDynamicSchema; + } + public TStorageFormat getStorageFormat() { // Force convert all V1 table to V2 table if (TStorageFormat.V1 == storageFormat) { @@ -327,6 +338,7 @@ public static TableProperty read(DataInput in) throws IOException { TableProperty tableProperty = GsonUtils.GSON.fromJson(Text.readString(in), TableProperty.class) .executeBuildDynamicProperty() .buildInMemory() + .buildDynamicSchema() .buildStorageFormat() .buildDataSortInfo() .buildCompressionType() diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/FeNameFormat.java b/fe/fe-core/src/main/java/org/apache/doris/common/FeNameFormat.java index b806cfe2711682..2a2c61abd034b3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/FeNameFormat.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/FeNameFormat.java @@ -28,7 +28,7 @@ public class FeNameFormat { private static final String LABEL_REGEX = "^[-_A-Za-z0-9]{1,128}$"; private static final String COMMON_NAME_REGEX = "^[a-zA-Z][a-zA-Z0-9_]{0,63}$"; private static final String COMMON_TABLE_NAME_REGEX = "^[a-zA-Z][a-zA-Z0-9_]*$"; - private static final String COLUMN_NAME_REGEX = "^[_a-zA-Z@0-9][a-zA-Z0-9_]{0,255}$"; + private static final String COLUMN_NAME_REGEX = "^[_a-zA-Z@0-9][.a-zA-Z0-9_+-/>"); + TYPE_STRING_MAP.put(PrimitiveType.ARRAY, "array<%s>"); + TYPE_STRING_MAP.put(PrimitiveType.VARIANT, "variant"); TYPE_STRING_MAP.put(PrimitiveType.NULL_TYPE, "null"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index eeceedf788b27f..6ceb1be9514757 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -277,6 +277,16 @@ public Database getDbNullable(long dbId) { return idToDb.get(dbId); } + public TableName getTableNameByTableId(Long tableId) { + for (Database db : fullNameToDb.values()) { + Table table = db.getTableNullable(tableId); + if (table != null) { + return new TableName("", db.getFullName(), table.getName()); + } + } + return null; + } + @Override public Map getProperties() { return Maps.newHashMap(); @@ -1364,6 +1374,10 @@ public void addPartition(Database db, String tableName, AddPartitionClause addPa properties.put(PropertyAnalyzer.PROPERTIES_STORE_ROW_COLUMN, olapTable.storeRowColumn().toString()); } + if (!properties.containsKey(PropertyAnalyzer.PROPERTIES_DYNAMIC_SCHEMA)) { + properties.put(PropertyAnalyzer.PROPERTIES_DYNAMIC_SCHEMA, + olapTable.isDynamicSchema().toString()); + } singlePartitionDesc.analyze(partitionInfo.getPartitionColumns().size(), properties); partitionInfo.createAndCheckPartitionItem(singlePartitionDesc, isTempPartition); @@ -1443,7 +1457,7 @@ public void addPartition(Database db, String tableName, AddPartitionClause addPa olapTable.getCopiedIndexes(), singlePartitionDesc.isInMemory(), olapTable.getStorageFormat(), singlePartitionDesc.getTabletType(), olapTable.getCompressionType(), olapTable.getDataSortInfo(), olapTable.getEnableUniqueKeyMergeOnWrite(), olapTable.getStoragePolicy(), idGeneratorBuffer, - olapTable.disableAutoCompaction(), olapTable.storeRowColumn()); + olapTable.disableAutoCompaction(), olapTable.storeRowColumn(), olapTable.isDynamicSchema()); // check again table = db.getOlapTableOrDdlException(tableName); @@ -1676,7 +1690,7 @@ private Partition createPartitionWithIndices(String clusterName, long dbId, long boolean isInMemory, TStorageFormat storageFormat, TTabletType tabletType, TCompressionType compressionType, DataSortInfo dataSortInfo, boolean enableUniqueKeyMergeOnWrite, String storagePolicy, IdGeneratorBuffer idGeneratorBuffer, boolean disableAutoCompaction, - boolean storeRowColumn) throws DdlException { + boolean storeRowColumn, boolean isDynamicSchema) throws DdlException { // create base index first. Preconditions.checkArgument(baseIndexId != -1); MaterializedIndex baseIndex = new MaterializedIndex(baseIndexId, IndexState.NORMAL); @@ -1738,7 +1752,7 @@ private Partition createPartitionWithIndices(String clusterName, long dbId, long tabletId, replicaId, shortKeyColumnCount, schemaHash, version, keysType, storageType, storageMedium, schema, bfColumns, bfFpp, countDownLatch, indexes, isInMemory, tabletType, dataSortInfo, compressionType, enableUniqueKeyMergeOnWrite, storagePolicy, - disableAutoCompaction, storeRowColumn); + disableAutoCompaction, storeRowColumn, isDynamicSchema); task.setStorageFormat(storageFormat); batchTask.addTask(task); @@ -1965,6 +1979,11 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep } olapTable.setStoreRowColumn(storeRowColumn); + // set dynamic schema + boolean isDynamicSchema = PropertyAnalyzer.analyzeBooleanProp(properties, + PropertyAnalyzer.PROPERTIES_DYNAMIC_SCHEMA, false); + olapTable.setIsDynamicSchema(isDynamicSchema); + // set storage policy String storagePolicy = PropertyAnalyzer.analyzeStoragePolicy(properties); Env.getCurrentEnv().getPolicyMgr().checkStoragePolicyExist(storagePolicy); @@ -2137,7 +2156,7 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep partitionInfo.getReplicaAllocation(partitionId), versionInfo, bfColumns, bfFpp, tabletIdSet, olapTable.getCopiedIndexes(), isInMemory, storageFormat, tabletType, compressionType, olapTable.getDataSortInfo(), olapTable.getEnableUniqueKeyMergeOnWrite(), storagePolicy, - idGeneratorBuffer, olapTable.disableAutoCompaction(), storeRowColumn); + idGeneratorBuffer, olapTable.disableAutoCompaction(), storeRowColumn, isDynamicSchema); olapTable.addPartition(partition); } else if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) { @@ -2196,7 +2215,7 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep tabletIdSet, olapTable.getCopiedIndexes(), isInMemory, storageFormat, partitionInfo.getTabletType(entry.getValue()), compressionType, olapTable.getDataSortInfo(), olapTable.getEnableUniqueKeyMergeOnWrite(), storagePolicy, - idGeneratorBuffer, olapTable.disableAutoCompaction(), storeRowColumn); + idGeneratorBuffer, olapTable.disableAutoCompaction(), storeRowColumn, isDynamicSchema); olapTable.addPartition(partition); } } else { @@ -2601,7 +2620,7 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti copiedTbl.getPartitionInfo().getTabletType(oldPartitionId), copiedTbl.getCompressionType(), copiedTbl.getDataSortInfo(), copiedTbl.getEnableUniqueKeyMergeOnWrite(), olapTable.getStoragePolicy(), idGeneratorBuffer, olapTable.disableAutoCompaction(), - olapTable.storeRowColumn()); + olapTable.storeRowColumn(), olapTable.isDynamicSchema()); newPartitions.add(newPartition); } } catch (DdlException e) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java index 3dda0456c1ffca..bfa59b281c5aab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java @@ -720,20 +720,46 @@ private static void initColumns(Table tbl, List columnExprs, slotDesc.setIsNullable(tblColumn.isAllowNull()); } } else { - // columns default be varchar type - slotDesc.setType(ScalarType.createType(PrimitiveType.VARCHAR)); - slotDesc.setColumn(new Column(realColName, PrimitiveType.VARCHAR)); - // ISSUE A: src slot should be nullable even if the column is not nullable. - // because src slot is what we read from file, not represent to real column value. - // If column is not nullable, error will be thrown when filling the dest slot, - // which is not nullable. - slotDesc.setIsNullable(true); + if (formatType == TFileFormatType.FORMAT_JSON + && tbl instanceof OlapTable && ((OlapTable) tbl).isDynamicSchema()) { + slotDesc.setType(tblColumn.getType()); + slotDesc.setColumn(new Column(realColName, tblColumn.getType())); + slotDesc.setIsNullable(tblColumn.isAllowNull()); + } else { + // columns default be varchar type + slotDesc.setType(ScalarType.createType(PrimitiveType.VARCHAR)); + slotDesc.setColumn(new Column(realColName, PrimitiveType.VARCHAR)); + // ISSUE A: src slot should be nullable even if the column is not nullable. + // because src slot is what we read from file, not represent to real column value. + // If column is not nullable, error will be thrown when filling the dest slot, + // which is not nullable. + slotDesc.setIsNullable(true); + } } slotDesc.setIsMaterialized(true); srcSlotIds.add(slotDesc.getId().asInt()); slotDescByName.put(realColName, slotDesc); } } + + // add a implict container column "DORIS_DYNAMIC_COL" for dynamic columns + if (tbl instanceof OlapTable && ((OlapTable) tbl).isDynamicSchema()) { + analyzer.getDescTbl().addReferencedTable(tbl); + SlotDescriptor slotDesc = analyzer.getDescTbl().addSlotDescriptor(srcTupleDesc); + String name = Column.DYNAMIC_COLUMN_NAME; + Column col = new Column(name, Type.VARIANT, false, null, false, "", + "stream load auto dynamic column"); + slotDesc.setType(Type.VARIANT); + slotDesc.setColumn(col); + // alaways nullable + slotDesc.setIsNullable(true); + slotDesc.setIsMaterialized(true); + srcSlotIds.add(slotDesc.getId().asInt()); + slotDescByName.put(name, slotDesc); + LOG.debug("add dynamic column to srcTupleDesc with name:{} id:{}", name, slotDesc.getId().asInt()); + } + LOG.debug("plan srcTupleDesc {}", srcTupleDesc.toString()); + /* * The extension column of the materialized view is added to the expression evaluation of load * To avoid nested expressions. eg : column(a, tmp_c, c = expr(tmp_c)) , diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java index dad51c44f7c9c0..142a8b4d15bf90 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java @@ -27,6 +27,7 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Type; import org.apache.doris.common.Config; import org.apache.doris.common.LoadException; import org.apache.doris.common.MetaNotFoundException; @@ -143,6 +144,22 @@ public void plan(TUniqueId loadId, List> fileStatusesLis } } + if (table.isDynamicSchema()) { + // Dynamic table for s3load ... + descTable.addReferencedTable(table); + // For reference table + scanTupleDesc.setTableId((int) table.getId()); + // Add a implict container column "DORIS_DYNAMIC_COL" for dynamic columns + SlotDescriptor slotDesc = descTable.addSlotDescriptor(scanTupleDesc); + Column col = new Column(Column.DYNAMIC_COLUMN_NAME, Type.VARIANT, false, null, false, "", + "stream load auto dynamic column"); + slotDesc.setIsMaterialized(true); + slotDesc.setColumn(col); + // alaways nullable + slotDesc.setIsNullable(true); + LOG.debug("plan scanTupleDesc{}", scanTupleDesc.toString()); + } + // Generate plan trees // 1. Broker scan node ScanNode scanNode; diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java index d45d787ed0549d..5ed52287c4d6ce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java @@ -773,7 +773,7 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta olapTable.getCompressionType(), olapTable.getEnableUniqueKeyMergeOnWrite(), olapTable.getStoragePolicy(), olapTable.disableAutoCompaction(), - olapTable.storeRowColumn()); + olapTable.storeRowColumn(), olapTable.isDynamicSchema()); createReplicaTask.setIsRecoverTask(true); createReplicaBatchTask.addTask(createReplicaTask); diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapTableSink.java index a390e4ecd7471a..a3e354384ec176 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapTableSink.java @@ -201,6 +201,7 @@ private TOlapTableSchemaParam createSchema(long dbId, OlapTable table) { schemaParam.setDbId(dbId); schemaParam.setTableId(table.getId()); schemaParam.setVersion(table.getIndexMetaByIndexId(table.getBaseIndexId()).getSchemaVersion()); + schemaParam.setIsDynamicSchema(table.isDynamicSchema()); schemaParam.tuple_desc = tupleDescriptor.toThrift(); for (SlotDescriptor slotDesc : tupleDescriptor.getSlots()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java index b8a0074d26454b..8e1bd99ab35535 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java @@ -36,6 +36,7 @@ import org.apache.doris.catalog.PartitionInfo; import org.apache.doris.catalog.PartitionItem; import org.apache.doris.catalog.PartitionType; +import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; @@ -169,6 +170,24 @@ public TExecPlanFragmentParams plan(TUniqueId loadId) throws UserException { } } + // Plan scan tuple of dynamic table + if (destTable.isDynamicSchema()) { + if (!Config.enable_vectorized_load) { + throw new UserException("Only support vectorized load for dyanmic table: " + destTable.getName()); + } + descTable.addReferencedTable(destTable); + scanTupleDesc.setTable(destTable); + // add a implict container column "DORIS_DYNAMIC_COL" for dynamic columns + SlotDescriptor slotDesc = descTable.addSlotDescriptor(scanTupleDesc); + Column col = new Column(Column.DYNAMIC_COLUMN_NAME, Type.VARIANT, false, null, false, "", + "stream load auto dynamic column"); + slotDesc.setIsMaterialized(true); + slotDesc.setColumn(col); + // alaways nullable + slotDesc.setIsNullable(true); + LOG.debug("plan tupleDesc {}", scanTupleDesc.toString()); + } + // create scan node ExternalFileScanNode fileScanNode = new ExternalFileScanNode(new PlanNodeId(0), scanTupleDesc); // 1. create file group diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index 19dde33b652a03..0789a2b72c7c65 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -18,13 +18,19 @@ package org.apache.doris.service; import org.apache.doris.alter.DecommissionType; +import org.apache.doris.alter.SchemaChangeHandler; +import org.apache.doris.analysis.AddColumnsClause; +import org.apache.doris.analysis.ColumnDef; import org.apache.doris.analysis.SetType; +import org.apache.doris.analysis.TableName; +import org.apache.doris.analysis.TypeDef; import org.apache.doris.analysis.UserIdentity; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.HMSResource; +import org.apache.doris.catalog.Index; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.TableIf; @@ -64,7 +70,10 @@ import org.apache.doris.task.StreamLoadTask; import org.apache.doris.thrift.FrontendService; import org.apache.doris.thrift.FrontendServiceVersion; +import org.apache.doris.thrift.TAddColumnsRequest; +import org.apache.doris.thrift.TAddColumnsResult; import org.apache.doris.thrift.TCell; +import org.apache.doris.thrift.TColumn; import org.apache.doris.thrift.TColumnDef; import org.apache.doris.thrift.TColumnDesc; import org.apache.doris.thrift.TDescribeTableParams; @@ -140,12 +149,17 @@ import java.time.Instant; import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.util.ArrayList; import java.util.HashMap; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.function.IntSupplier; // Frontend service used to serve all request for this frontend through // thrift protocol @@ -214,6 +228,161 @@ public TGetDbsResult getDbNames(TGetDbsParams params) throws TException { return result; } + private static ColumnDef initColumnfromThrift(TColumnDesc tColumnDesc, String comment) { + TypeDef typeDef = TypeDef.createTypeDef(tColumnDesc); + boolean isAllowNull = tColumnDesc.isIsAllowNull(); + ColumnDef.DefaultValue defaultVal = ColumnDef.DefaultValue.NOT_SET; + // Dynamic table's Array default value should be '[]' + if (typeDef.getType().isArrayType()) { + defaultVal = ColumnDef.DefaultValue.ARRAY_EMPTY_DEFAULT_VALUE; + } + return new ColumnDef(tColumnDesc.getColumnName(), typeDef, false, null, isAllowNull, defaultVal, + comment, true); + } + + @Override + public TAddColumnsResult addColumns(TAddColumnsRequest request) throws TException { + String clientAddr = getClientAddrAsString(); + LOG.debug("schema change clientAddr: {}, request: {}", clientAddr, request); + + TStatus status = new TStatus(TStatusCode.OK); + List allColumns = new ArrayList(); + + Env env = Env.getCurrentEnv(); + InternalCatalog catalog = env.getInternalCatalog(); + int schemaVersion = 0; + try { + if (!env.isMaster()) { + status.setStatusCode(TStatusCode.ILLEGAL_STATE); + status.addToErrorMsgs("retry rpc request to master."); + TAddColumnsResult result = new TAddColumnsResult(status, request.getTableId(), allColumns, 0); + LOG.debug("result: {}", result); + return result; + } + TableName tableName = new TableName("", request.getDbName(), request.getTableName()); + if (request.getTableId() > 0) { + tableName = catalog.getTableNameByTableId(request.getTableId()); + } + if (tableName == null) { + throw new MetaNotFoundException("table_id " + request.getTableId() + " does not exist"); + } + + Database db = catalog.getDbNullable(tableName.getDb()); + if (db == null) { + throw new MetaNotFoundException("db " + tableName.getDb() + " does not exist"); + } + + List addColumns = request.getAddColumns(); + boolean queryMode = false; + if (addColumns == null || addColumns.size() == 0) { + queryMode = true; + } + + // rpc only olap table + OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableName.getTbl(), TableType.OLAP); + olapTable.writeLockOrMetaException(); + + try { + List columnDefs = new ArrayList(); + + // prepare columnDefs + for (TColumnDef tColumnDef : addColumns) { + if (request.isAllowTypeConflict()) { + // ignore column with same name + boolean hasSameNameColumn = false; + for (Column column : olapTable.getBaseSchema()) { + if (column.getName().equals(tColumnDef.getColumnDesc().getColumnName())) { + hasSameNameColumn = true; + } + } + // ignore this column + if (hasSameNameColumn) { + continue; + } + } + String comment = tColumnDef.getComment(); + if (comment == null || comment.length() == 0) { + Instant ins = Instant.ofEpochSecond(System.currentTimeMillis() / 1000); + ZonedDateTime zdt = ins.atZone(ZoneId.systemDefault()); + comment = "auto change " + zdt.toString(); + } + + TColumnDesc tColumnDesc = tColumnDef.getColumnDesc(); + ColumnDef columnDef = initColumnfromThrift(tColumnDesc, comment); + columnDefs.add(columnDef); + } + + if (!queryMode && !columnDefs.isEmpty()) { + // create AddColumnsClause + AddColumnsClause addColumnsClause = new AddColumnsClause(columnDefs, null, null); + addColumnsClause.analyze(null); + + // index id -> index schema + Map> indexSchemaMap = new HashMap<>(); + //index id -> index col_unique_id supplier + Map colUniqueIdSupplierMap = new HashMap<>(); + for (Map.Entry> entry : olapTable.getIndexIdToSchema(true).entrySet()) { + indexSchemaMap.put(entry.getKey(), new LinkedList<>(entry.getValue())); + IntSupplier colUniqueIdSupplier = null; + if (olapTable.getEnableLightSchemaChange()) { + colUniqueIdSupplier = new IntSupplier() { + public int pendingMaxColUniqueId = olapTable + .getIndexMetaByIndexId(entry.getKey()).getMaxColUniqueId(); + @Override + public int getAsInt() { + pendingMaxColUniqueId++; + return pendingMaxColUniqueId; + } + }; + } + colUniqueIdSupplierMap.put(entry.getKey(), colUniqueIdSupplier); + } + //4. call schame change function, only for dynamic table feature. + SchemaChangeHandler schemaChangeHandler = new SchemaChangeHandler(); + + boolean lightSchemaChange = schemaChangeHandler.processAddColumns( + addColumnsClause, olapTable, indexSchemaMap, true, colUniqueIdSupplierMap); + if (lightSchemaChange) { + //for schema change add column optimize, direct modify table meta. + List newIndexes = olapTable.getCopiedIndexes(); + long jobId = Env.getCurrentEnv().getNextId(); + Env.getCurrentEnv().getSchemaChangeHandler().modifyTableAddOrDropColumns( + db, olapTable, indexSchemaMap, newIndexes, jobId, false); + } else { + throw new MetaNotFoundException("table_id " + + request.getTableId() + " cannot light schema change through rpc."); + } + } + + //5. build all columns + for (Column column : olapTable.getBaseSchema()) { + allColumns.add(column.toThrift()); + } + schemaVersion = olapTable.getBaseSchemaVersion(); + } catch (Exception e) { + LOG.warn("got exception add columns: ", e); + status.setStatusCode(TStatusCode.INTERNAL_ERROR); + status.addToErrorMsgs(e.getMessage()); + } finally { + olapTable.writeUnlock(); + } + } catch (MetaNotFoundException e) { + status.setStatusCode(TStatusCode.NOT_FOUND); + status.addToErrorMsgs(e.getMessage()); + } catch (UserException e) { + status.setStatusCode(TStatusCode.INVALID_ARGUMENT); + status.addToErrorMsgs(e.getMessage()); + } catch (Exception e) { + LOG.warn("got exception add columns: ", e); + status.setStatusCode(TStatusCode.INTERNAL_ERROR); + status.addToErrorMsgs(e.getMessage()); + } + + TAddColumnsResult result = new TAddColumnsResult(status, request.getTableId(), allColumns, schemaVersion); + LOG.debug("result: {}", result); + return result; + } + @Override public TGetTablesResult getTableNames(TGetTablesParams params) throws TException { LOG.debug("get table name request: {}", params); diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java index fcb139fd8f7d0b..ddf7d79d892ad3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java @@ -73,6 +73,8 @@ public class CreateReplicaTask extends AgentTask { private boolean isInMemory; + private boolean isDynamicSchema; + private TTabletType tabletType; // used for synchronous process @@ -112,7 +114,8 @@ public CreateReplicaTask(long backendId, long dbId, long tableId, long partition TCompressionType compressionType, boolean enableUniqueKeyMergeOnWrite, String storagePolicy, boolean disableAutoCompaction, - boolean storeRowColumn) { + boolean storeRowColumn, + boolean isDynamicSchema) { super(null, backendId, TTaskType.CREATE, dbId, tableId, partitionId, indexId, tabletId); this.replicaId = replicaId; @@ -135,6 +138,7 @@ public CreateReplicaTask(long backendId, long dbId, long tableId, long partition this.latch = latch; this.isInMemory = isInMemory; + this.isDynamicSchema = isDynamicSchema; this.tabletType = tabletType; this.dataSortInfo = dataSortInfo; this.enableUniqueKeyMergeOnWrite = (keysType == KeysType.UNIQUE_KEYS && enableUniqueKeyMergeOnWrite); @@ -247,6 +251,7 @@ public TCreateTabletReq toThrift() { tSchema.setIsInMemory(isInMemory); tSchema.setDisableAutoCompaction(disableAutoCompaction); tSchema.setStoreRowColumn(storeRowColumn); + tSchema.setIsDynamicSchema(isDynamicSchema); createTabletReq.setTabletSchema(tSchema); createTabletReq.setVersion(version); diff --git a/fe/fe-core/src/test/java/org/apache/doris/common/FeNameFormatTest.java b/fe/fe-core/src/test/java/org/apache/doris/common/FeNameFormatTest.java index eeca51fbf45fab..b952ab82e67031 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/common/FeNameFormatTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/common/FeNameFormatTest.java @@ -28,10 +28,12 @@ public void testCheckColumnName() { ExceptionChecker.expectThrowsNoException(() -> FeNameFormat.checkColumnName("___id")); ExceptionChecker.expectThrowsNoException(() -> FeNameFormat.checkColumnName("___id_")); ExceptionChecker.expectThrowsNoException(() -> FeNameFormat.checkColumnName("@timestamp")); + ExceptionChecker.expectThrowsNoException(() -> FeNameFormat.checkColumnName("@timestamp#")); + ExceptionChecker.expectThrowsNoException(() -> FeNameFormat.checkColumnName("timestamp*")); + ExceptionChecker.expectThrowsNoException(() -> FeNameFormat.checkColumnName("timestamp.1")); + ExceptionChecker.expectThrowsNoException(() -> FeNameFormat.checkColumnName("timestamp.#")); ExceptionChecker.expectThrows(AnalysisException.class, () -> FeNameFormat.checkColumnName("?id_")); ExceptionChecker.expectThrows(AnalysisException.class, () -> FeNameFormat.checkColumnName("#id_")); - ExceptionChecker.expectThrows(AnalysisException.class, () -> FeNameFormat.checkColumnName("@@timestamp")); - ExceptionChecker.expectThrows(AnalysisException.class, () -> FeNameFormat.checkColumnName("@timestamp@")); // length 64 String tblName = "test_sys_partition_list_basic_test_list_partition_bigint_tb_uniq"; ExceptionChecker.expectThrowsNoException(() -> FeNameFormat.checkTableName(tblName)); diff --git a/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java index 5ff129b34b8f1d..fc03ee92896ab3 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java @@ -107,7 +107,7 @@ public void setUp() throws AnalysisException { createReplicaTask = new CreateReplicaTask(backendId1, dbId, tableId, partitionId, indexId1, tabletId1, replicaId1, shortKeyNum, schemaHash1, version, KeysType.AGG_KEYS, storageType, TStorageMedium.SSD, columns, null, 0, latch, null, false, TTabletType.TABLET_TYPE_DISK, null, - TCompressionType.LZ4F, false, "", false, false); + TCompressionType.LZ4F, false, "", false, false, false); // drop dropTask = new DropReplicaTask(backendId1, tabletId1, replicaId1, schemaHash1, false); diff --git a/gensrc/proto/olap_file.proto b/gensrc/proto/olap_file.proto index 4e65af4f065a29..16b7cf497d5fce 100644 --- a/gensrc/proto/olap_file.proto +++ b/gensrc/proto/olap_file.proto @@ -229,6 +229,7 @@ message TabletSchemaPB { optional bool disable_auto_compaction = 15 [default=false]; repeated TabletIndexPB index = 16; optional bool store_row_column = 17 [default=false]; // store tuplerow oriented column + optional bool is_dynamic_schema = 18 [default=false]; } enum TabletStatePB { diff --git a/gensrc/proto/types.proto b/gensrc/proto/types.proto index dea020d0b4d03b..c97dc64b4c01c9 100644 --- a/gensrc/proto/types.proto +++ b/gensrc/proto/types.proto @@ -105,6 +105,7 @@ message PGenericType { FIXEDLENGTHOBJECT = 30; JSONB = 31; DECIMAL128I = 32; + VARIANT = 33; UNKNOWN = 999; } required TypeId id = 2; diff --git a/gensrc/thrift/AgentService.thrift b/gensrc/thrift/AgentService.thrift index 4c8b61cf5a200f..831f6c3b9433eb 100644 --- a/gensrc/thrift/AgentService.thrift +++ b/gensrc/thrift/AgentService.thrift @@ -40,6 +40,7 @@ struct TTabletSchema { 12: optional i32 sort_col_num 13: optional bool disable_auto_compaction 14: optional bool store_row_column = false + 15: optional bool is_dynamic_schema = false } // this enum stands for different storage format in src_backends diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift index d8e0cc09fa382d..6d3c2e93c05611 100644 --- a/gensrc/thrift/Descriptors.thrift +++ b/gensrc/thrift/Descriptors.thrift @@ -199,6 +199,7 @@ struct TOlapTableSchemaParam { 4: required list slot_descs 5: required TTupleDescriptor tuple_desc 6: required list indexes + 7: optional bool is_dynamic_schema } struct TOlapTableIndex { diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 457e78020db64f..be331f52038d35 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -50,6 +50,7 @@ struct TColumnDesc { 5: optional i32 columnScale 6: optional bool isAllowNull 7: optional string columnKey + 8: optional list children } // A column definition; used by CREATE TABLE and DESCRIBE statements. A column @@ -716,6 +717,23 @@ struct TFetchSchemaTableDataResult { 2: optional list data_batch; } +// Only support base table add columns +struct TAddColumnsRequest { + 1: required i64 table_id + 2: required list addColumns + 3: optional string table_name + 4: optional string db_name + 5: optional bool allow_type_conflict +} + +// Only support base table add columns +struct TAddColumnsResult { + 1: required Status.TStatus status + 2: required i64 table_id + 3: required list allColumns + 4: required i32 schema_version +} + service FrontendService { TGetDbsResult getDbNames(1: TGetDbsParams params) TGetTablesResult getTableNames(1: TGetTablesParams params) @@ -750,6 +768,8 @@ service FrontendService { TFrontendPingFrontendResult ping(1: TFrontendPingFrontendRequest request) + TAddColumnsResult addColumns(1: TAddColumnsRequest request) + TInitExternalCtlMetaResult initExternalCtlMeta(1: TInitExternalCtlMetaRequest request) TFetchSchemaTableDataResult fetchSchemaTableData(1: TFetchSchemaTableDataRequest request) diff --git a/gensrc/thrift/Types.thrift b/gensrc/thrift/Types.thrift index d6f05a0be2d24e..d1c9304eb7a111 100644 --- a/gensrc/thrift/Types.thrift +++ b/gensrc/thrift/Types.thrift @@ -91,6 +91,7 @@ enum TPrimitiveType { DECIMAL64, DECIMAL128I, JSONB, + VARIANT, UNSUPPORTED } @@ -98,7 +99,8 @@ enum TTypeNodeType { SCALAR, ARRAY, MAP, - STRUCT + STRUCT, + VARIANT, } enum TStorageBackendType {