diff --git a/be/src/agent/heartbeat_server.cpp b/be/src/agent/heartbeat_server.cpp index 326758ffeb486c..964a898129b09f 100644 --- a/be/src/agent/heartbeat_server.cpp +++ b/be/src/agent/heartbeat_server.cpp @@ -76,6 +76,7 @@ void HeartbeatServer::heartbeat(THeartbeatResult& heartbeat_result, heartbeat_result.backend_info.__set_http_port(config::webserver_port); heartbeat_result.backend_info.__set_be_rpc_port(-1); heartbeat_result.backend_info.__set_brpc_port(config::brpc_port); + heartbeat_result.backend_info.__set_arrow_flight_sql_port(config::arrow_flight_sql_port); heartbeat_result.backend_info.__set_version(get_short_version()); heartbeat_result.backend_info.__set_be_start_time(_be_epoch); heartbeat_result.backend_info.__set_be_node_role(config::be_node_role); diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index cf3b8cd80ccf8f..389a23d77b5d3d 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -59,7 +59,7 @@ DEFINE_Int32(be_port, "9060"); // port for brpc DEFINE_Int32(brpc_port, "8060"); -DEFINE_Int32(arrow_flight_port, "-1"); +DEFINE_Int32(arrow_flight_sql_port, "-1"); // the number of bthreads for brpc, the default value is set to -1, // which means the number of bthreads is #cpu-cores @@ -581,7 +581,7 @@ DEFINE_mInt32(result_buffer_cancelled_interval_time, "300"); DEFINE_mInt32(priority_queue_remaining_tasks_increased_frequency, "512"); // sync tablet_meta when modifying meta -DEFINE_mBool(sync_tablet_meta, "false"); +DEFINE_mBool(sync_tablet_meta, "true"); // default thrift rpc timeout ms DEFINE_mInt32(thrift_rpc_timeout_ms, "60000"); diff --git a/be/src/common/config.h b/be/src/common/config.h index 4c41e6f0e16423..b34a6c98ccda5b 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -96,9 +96,9 @@ DECLARE_Int32(be_port); // port for brpc DECLARE_Int32(brpc_port); -// port for arrow flight -// Default -1, do not start arrow flight server. -DECLARE_Int32(arrow_flight_port); +// port for arrow flight sql +// Default -1, do not start arrow flight sql server. +DECLARE_Int32(arrow_flight_sql_port); // the number of bthreads for brpc, the default value is set to -1, // which means the number of bthreads is #cpu-cores diff --git a/be/src/exec/data_sink.cpp b/be/src/exec/data_sink.cpp index d5af0cff62055d..c63137420ed6b4 100644 --- a/be/src/exec/data_sink.cpp +++ b/be/src/exec/data_sink.cpp @@ -31,7 +31,6 @@ #include "common/config.h" #include "vec/sink/async_writer_sink.h" -#include "vec/sink/group_commit_vtablet_sink.h" #include "vec/sink/multi_cast_data_stream_sink.h" #include "vec/sink/vdata_stream_sender.h" #include "vec/sink/vmemory_scratch_sink.h" @@ -146,21 +145,20 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink break; } case TDataSinkType::OLAP_TABLE_SINK: { - Status status; + Status status = Status::OK(); DCHECK(thrift_sink.__isset.olap_table_sink); if (state->query_options().enable_memtable_on_sink_node) { - sink->reset(new stream_load::VOlapTableSinkV2(pool, row_desc, output_exprs, &status)); + sink->reset(new vectorized::VOlapTableSinkV2(pool, row_desc, output_exprs, &status)); } else { - sink->reset(new stream_load::VOlapTableSink(pool, row_desc, output_exprs, &status)); + sink->reset(new vectorized::VOlapTableSink(pool, row_desc, output_exprs, false)); } RETURN_IF_ERROR(status); break; } case TDataSinkType::GROUP_COMMIT_OLAP_TABLE_SINK: { - Status status; + Status status = Status::OK(); DCHECK(thrift_sink.__isset.olap_table_sink); - sink->reset( - new stream_load::GroupCommitVOlapTableSink(pool, row_desc, output_exprs, &status)); + sink->reset(new vectorized::VOlapTableSink(pool, row_desc, output_exprs, true)); RETURN_IF_ERROR(status); break; } @@ -294,12 +292,12 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink break; } case TDataSinkType::OLAP_TABLE_SINK: { - Status status; + Status status = Status::OK(); DCHECK(thrift_sink.__isset.olap_table_sink); if (state->query_options().enable_memtable_on_sink_node) { - sink->reset(new stream_load::VOlapTableSinkV2(pool, row_desc, output_exprs, &status)); + sink->reset(new vectorized::VOlapTableSinkV2(pool, row_desc, output_exprs, &status)); } else { - sink->reset(new stream_load::VOlapTableSink(pool, row_desc, output_exprs, &status)); + sink->reset(new vectorized::VOlapTableSink(pool, row_desc, output_exprs, false)); } RETURN_IF_ERROR(status); break; @@ -313,10 +311,9 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink break; } case TDataSinkType::GROUP_COMMIT_OLAP_TABLE_SINK: { - Status status; + Status status = Status::OK(); DCHECK(thrift_sink.__isset.olap_table_sink); - sink->reset( - new stream_load::GroupCommitVOlapTableSink(pool, row_desc, output_exprs, &status)); + sink->reset(new vectorized::VOlapTableSink(pool, row_desc, output_exprs, true)); RETURN_IF_ERROR(status); break; } diff --git a/be/src/exec/odbc_connector.cpp b/be/src/exec/odbc_connector.cpp index 8a74b7a27cf899..25ce819ca3c733 100644 --- a/be/src/exec/odbc_connector.cpp +++ b/be/src/exec/odbc_connector.cpp @@ -58,7 +58,7 @@ ODBCConnector::ODBCConnector(const ODBCConnectorParam& param) _dbc(nullptr), _stmt(nullptr) {} -Status ODBCConnector::close() { +Status ODBCConnector::close(Status) { // do not commit transaction, roll back if (_is_in_transaction) { abort_trans(); diff --git a/be/src/exec/odbc_connector.h b/be/src/exec/odbc_connector.h index 3618519958faf8..4edd1e879cb549 100644 --- a/be/src/exec/odbc_connector.h +++ b/be/src/exec/odbc_connector.h @@ -97,7 +97,7 @@ class ODBCConnector : public TableConnector { uint32_t big_column_size_buffer = config::big_column_size_buffer; uint32_t small_column_size_buffer = config::small_column_size_buffer; - Status close() override; + Status close(Status) override; private: static Status error_status(const std::string& prefix, const std::string& error_msg); diff --git a/be/src/exec/table_connector.h b/be/src/exec/table_connector.h index eb4292161d51c7..7a0dd63e06610e 100644 --- a/be/src/exec/table_connector.h +++ b/be/src/exec/table_connector.h @@ -55,7 +55,7 @@ class TableConnector { virtual Status abort_trans() = 0; // should be call after transaction abort virtual Status finish_trans() = 0; // should be call after transaction commit - virtual Status close() = 0; + virtual Status close(Status) = 0; virtual Status exec_stmt_write(vectorized::Block* block, const vectorized::VExprContextSPtrs& _output_vexpr_ctxs, diff --git a/be/src/pipeline/exec/olap_table_sink_operator.h b/be/src/pipeline/exec/olap_table_sink_operator.h index 34c276be3afa98..e3bab01faf4eac 100644 --- a/be/src/pipeline/exec/olap_table_sink_operator.h +++ b/be/src/pipeline/exec/olap_table_sink_operator.h @@ -25,7 +25,7 @@ namespace doris { namespace pipeline { class OlapTableSinkOperatorBuilder final - : public DataSinkOperatorBuilder { + : public DataSinkOperatorBuilder { public: OlapTableSinkOperatorBuilder(int32_t id, DataSink* sink) : DataSinkOperatorBuilder(id, "OlapTableSinkOperator", sink) {} diff --git a/be/src/pipeline/exec/olap_table_sink_v2_operator.h b/be/src/pipeline/exec/olap_table_sink_v2_operator.h index 75f56f6ba60047..f280e856f0ca17 100644 --- a/be/src/pipeline/exec/olap_table_sink_v2_operator.h +++ b/be/src/pipeline/exec/olap_table_sink_v2_operator.h @@ -25,7 +25,7 @@ namespace doris { namespace pipeline { class OlapTableSinkV2OperatorBuilder final - : public DataSinkOperatorBuilder { + : public DataSinkOperatorBuilder { public: OlapTableSinkV2OperatorBuilder(int32_t id, DataSink* sink) : DataSinkOperatorBuilder(id, "OlapTableSinkV2Operator", sink) {} diff --git a/be/src/pipeline/pipeline_task.cpp b/be/src/pipeline/pipeline_task.cpp index add92cabf37065..be0ecbffe741a4 100644 --- a/be/src/pipeline/pipeline_task.cpp +++ b/be/src/pipeline/pipeline_task.cpp @@ -122,6 +122,9 @@ void PipelineTask::_init_profile() { _schedule_counts = ADD_COUNTER(_task_profile, "NumScheduleTimes", TUnit::UNIT); _yield_counts = ADD_COUNTER(_task_profile, "NumYieldTimes", TUnit::UNIT); _core_change_times = ADD_COUNTER(_task_profile, "CoreChangeTimes", TUnit::UNIT); + _wait_bf_counts = ADD_COUNTER(_task_profile, "WaitBfTimes", TUnit::UNIT); + _wait_dependency_counts = ADD_COUNTER(_task_profile, "WaitDenpendencyTimes", TUnit::UNIT); + _pending_finish_counts = ADD_COUNTER(_task_profile, "PendingFinishTimes", TUnit::UNIT); _begin_execute_timer = ADD_TIMER(_task_profile, "Task1BeginExecuteTime"); _eos_timer = ADD_TIMER(_task_profile, "Task2EosTime"); @@ -385,6 +388,11 @@ void PipelineTask::set_state(PipelineTaskState state) { COUNTER_UPDATE(_block_by_sink_counts, 1); } else if (state == PipelineTaskState::BLOCKED_FOR_RF) { _wait_bf_watcher.start(); + COUNTER_UPDATE(_wait_bf_counts, 1); + } else if (state == PipelineTaskState::BLOCKED_FOR_DEPENDENCY) { + COUNTER_UPDATE(_wait_dependency_counts, 1); + } else if (state == PipelineTaskState::PENDING_FINISH) { + COUNTER_UPDATE(_pending_finish_counts, 1); } } diff --git a/be/src/pipeline/pipeline_task.h b/be/src/pipeline/pipeline_task.h index b8b8e89215f565..bc5ac96ae9816f 100644 --- a/be/src/pipeline/pipeline_task.h +++ b/be/src/pipeline/pipeline_task.h @@ -304,10 +304,13 @@ class PipelineTask { RuntimeProfile::Counter* _wait_source_timer; MonotonicStopWatch _wait_bf_watcher; RuntimeProfile::Counter* _wait_bf_timer; + RuntimeProfile::Counter* _wait_bf_counts; MonotonicStopWatch _wait_sink_watcher; RuntimeProfile::Counter* _wait_sink_timer; MonotonicStopWatch _wait_worker_watcher; RuntimeProfile::Counter* _wait_worker_timer; + RuntimeProfile::Counter* _wait_dependency_counts; + RuntimeProfile::Counter* _pending_finish_counts; // TODO we should calculate the time between when really runnable and runnable RuntimeProfile::Counter* _yield_counts; RuntimeProfile::Counter* _core_change_times; diff --git a/be/src/pipeline/pipeline_x/pipeline_x_task.cpp b/be/src/pipeline/pipeline_x/pipeline_x_task.cpp index 41d5226e85a88a..dcb3aa1b27b13a 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_task.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_task.cpp @@ -114,6 +114,10 @@ void PipelineXTask::_init_profile() { _schedule_counts = ADD_COUNTER(_task_profile, "NumScheduleTimes", TUnit::UNIT); _yield_counts = ADD_COUNTER(_task_profile, "NumYieldTimes", TUnit::UNIT); _core_change_times = ADD_COUNTER(_task_profile, "CoreChangeTimes", TUnit::UNIT); + + _wait_bf_counts = ADD_COUNTER(_task_profile, "WaitBfTimes", TUnit::UNIT); + _wait_dependency_counts = ADD_COUNTER(_task_profile, "WaitDenpendencyTimes", TUnit::UNIT); + _pending_finish_counts = ADD_COUNTER(_task_profile, "PendingFinishTimes", TUnit::UNIT); } void PipelineXTask::_fresh_profile_counter() { diff --git a/be/src/runtime/buffer_control_block.cpp b/be/src/runtime/buffer_control_block.cpp index c51ced1aeb2248..adbaf7fbb0d787 100644 --- a/be/src/runtime/buffer_control_block.cpp +++ b/be/src/runtime/buffer_control_block.cpp @@ -144,7 +144,6 @@ Status BufferControlBlock::add_batch(std::unique_ptr& result) _fe_result_batch_queue.push_back(std::move(result)); } _buffer_rows += num_rows; - _data_arrival.notify_one(); } else { auto ctx = _waiting_rpc.front(); _waiting_rpc.pop_front(); diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index d52e7cd3612f8a..2d339476ebfb78 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -42,6 +42,7 @@ namespace doris { namespace vectorized { class VDataStreamMgr; class ScannerScheduler; +class DeltaWriterV2Pool; using ZoneList = std::unordered_map; } // namespace vectorized namespace pipeline { @@ -51,7 +52,6 @@ namespace taskgroup { class TaskGroupManager; } namespace stream_load { -class DeltaWriterV2Pool; class LoadStreamStubPool; } // namespace stream_load namespace io { @@ -241,7 +241,8 @@ class ExecEnv { stream_load::LoadStreamStubPool* load_stream_stub_pool() { return _load_stream_stub_pool.get(); } - stream_load::DeltaWriterV2Pool* delta_writer_v2_pool() { return _delta_writer_v2_pool.get(); } + + vectorized::DeltaWriterV2Pool* delta_writer_v2_pool() { return _delta_writer_v2_pool.get(); } void wait_for_all_tasks_done(); @@ -344,7 +345,7 @@ class ExecEnv { FileMetaCache* _file_meta_cache = nullptr; std::unique_ptr _memtable_memory_limiter; std::unique_ptr _load_stream_stub_pool; - std::unique_ptr _delta_writer_v2_pool; + std::unique_ptr _delta_writer_v2_pool; std::unique_ptr _global_zone_cache; std::shared_mutex _zone_cache_rw_lock; diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 24234a90d957ce..975189ef828b50 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -210,7 +210,7 @@ Status ExecEnv::_init(const std::vector& store_paths) { _file_meta_cache = new FileMetaCache(config::max_external_file_meta_cache_num); _memtable_memory_limiter = std::make_unique(); _load_stream_stub_pool = std::make_unique(); - _delta_writer_v2_pool = std::make_unique(); + _delta_writer_v2_pool = std::make_unique(); _backend_client_cache->init_metrics("backend"); _frontend_client_cache->init_metrics("frontend"); diff --git a/be/src/runtime/result_buffer_mgr.cpp b/be/src/runtime/result_buffer_mgr.cpp index a3b99300f209c5..c4d0f148ed2a23 100644 --- a/be/src/runtime/result_buffer_mgr.cpp +++ b/be/src/runtime/result_buffer_mgr.cpp @@ -21,6 +21,9 @@ #include #include #include + +#include +#include // IWYU pragma: no_include #include // IWYU pragma: keep #include @@ -33,6 +36,7 @@ #include "util/doris_metrics.h" #include "util/metrics.h" #include "util/thread.h" +#include "util/uid_util.h" namespace doris { @@ -42,7 +46,7 @@ ResultBufferMgr::ResultBufferMgr() : _stop_background_threads_latch(1) { // Each BufferControlBlock has a limited queue size of 1024, it's not needed to count the // actual size of all BufferControlBlock. REGISTER_HOOK_METRIC(result_buffer_block_count, [this]() { - // std::lock_guard l(_lock); + // std::lock_guard l(_buffer_map_lock); return _buffer_map.size(); }); } @@ -80,7 +84,7 @@ Status ResultBufferMgr::create_sender(const TUniqueId& query_id, int buffer_size } { - std::lock_guard l(_lock); + std::unique_lock wlock(_buffer_map_lock); _buffer_map.insert(std::make_pair(query_id, control_block)); // BufferControlBlock should destroy after max_timeout // for exceed max_timeout FE will return timeout to client @@ -95,8 +99,7 @@ Status ResultBufferMgr::create_sender(const TUniqueId& query_id, int buffer_size } std::shared_ptr ResultBufferMgr::find_control_block(const TUniqueId& query_id) { - // TODO(zhaochun): this lock can be bottleneck? - std::lock_guard l(_lock); + std::shared_lock rlock(_buffer_map_lock); BufferMap::iterator iter = _buffer_map.find(query_id); if (_buffer_map.end() != iter) { @@ -108,14 +111,12 @@ std::shared_ptr ResultBufferMgr::find_control_block(const TU void ResultBufferMgr::register_row_descriptor(const TUniqueId& query_id, const RowDescriptor& row_desc) { - { - std::lock_guard l(_lock); - _row_descriptor_map.insert(std::make_pair(query_id, row_desc)); - } + std::unique_lock wlock(_row_descriptor_map_lock); + _row_descriptor_map.insert(std::make_pair(query_id, row_desc)); } RowDescriptor ResultBufferMgr::find_row_descriptor(const TUniqueId& query_id) { - std::lock_guard l(_lock); + std::shared_lock rlock(_row_descriptor_map_lock); RowDescriptorMap::iterator iter = _row_descriptor_map.find(query_id); if (_row_descriptor_map.end() != iter) { @@ -150,18 +151,23 @@ Status ResultBufferMgr::fetch_arrow_data(const TUniqueId& finst_id, } Status ResultBufferMgr::cancel(const TUniqueId& query_id) { - std::lock_guard l(_lock); - BufferMap::iterator iter = _buffer_map.find(query_id); + { + std::unique_lock wlock(_buffer_map_lock); + BufferMap::iterator iter = _buffer_map.find(query_id); - if (_buffer_map.end() != iter) { - iter->second->cancel(); - _buffer_map.erase(iter); + if (_buffer_map.end() != iter) { + iter->second->cancel(); + _buffer_map.erase(iter); + } } - RowDescriptorMap::iterator row_desc_iter = _row_descriptor_map.find(query_id); + { + std::unique_lock wlock(_row_descriptor_map_lock); + RowDescriptorMap::iterator row_desc_iter = _row_descriptor_map.find(query_id); - if (_row_descriptor_map.end() != row_desc_iter) { - _row_descriptor_map.erase(row_desc_iter); + if (_row_descriptor_map.end() != row_desc_iter) { + _row_descriptor_map.erase(row_desc_iter); + } } return Status::OK(); diff --git a/be/src/runtime/result_buffer_mgr.h b/be/src/runtime/result_buffer_mgr.h index 8c9b621968a296..4e5cd38a7264b7 100644 --- a/be/src/runtime/result_buffer_mgr.h +++ b/be/src/runtime/result_buffer_mgr.h @@ -23,6 +23,7 @@ #include #include #include +#include #include #include @@ -86,9 +87,11 @@ class ResultBufferMgr { void cancel_thread(); // lock for buffer map - std::mutex _lock; + std::shared_mutex _buffer_map_lock; // buffer block map BufferMap _buffer_map; + // lock for descriptor map + std::shared_mutex _row_descriptor_map_lock; // for arrow flight RowDescriptorMap _row_descriptor_map; diff --git a/be/src/runtime/result_writer.h b/be/src/runtime/result_writer.h index 01b9bb948b7dae..f199d2f6b9d21e 100644 --- a/be/src/runtime/result_writer.h +++ b/be/src/runtime/result_writer.h @@ -38,7 +38,7 @@ class ResultWriter { virtual Status init(RuntimeState* state) = 0; - virtual Status close() = 0; + virtual Status close(Status s = Status::OK()) = 0; [[nodiscard]] virtual int64_t get_written_rows() const { return _written_rows; } diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index e862ef27a63c19..5275a21c64d026 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -94,6 +94,7 @@ class RuntimeState { const TQueryGlobals& query_globals, ExecEnv* exec_env); // for ut and non-query. + void set_exec_env(ExecEnv* exec_env) { _exec_env = exec_env; } void init_mem_trackers(const TUniqueId& id = TUniqueId(), const std::string& name = "unknown"); const TQueryOptions& query_options() const { return _query_options; } diff --git a/be/src/service/arrow_flight/arrow_flight_batch_reader.cpp b/be/src/service/arrow_flight/arrow_flight_batch_reader.cpp index d7a648c54b6108..8a0f1e67859494 100644 --- a/be/src/service/arrow_flight/arrow_flight_batch_reader.cpp +++ b/be/src/service/arrow_flight/arrow_flight_batch_reader.cpp @@ -57,12 +57,17 @@ arrow::Result> ArrowFlightBatchReader::C } arrow::Status ArrowFlightBatchReader::ReadNext(std::shared_ptr* out) { - CHECK(*out == nullptr); + // *out not nullptr + *out = nullptr; auto st = ExecEnv::GetInstance()->result_mgr()->fetch_arrow_data(statement_->query_id, out); if (UNLIKELY(!st.ok())) { - LOG(WARNING) << st.to_string(); + LOG(WARNING) << "ArrowFlightBatchReader fetch arrow data failed: " + st.to_string(); ARROW_RETURN_NOT_OK(to_arrow_status(st)); } + if (*out != nullptr) { + VLOG_NOTICE << "ArrowFlightBatchReader read next: " << (*out)->num_rows() << ", " + << (*out)->num_columns(); + } return arrow::Status::OK(); } diff --git a/be/src/service/doris_main.cpp b/be/src/service/doris_main.cpp index abfa7913e7d541..3e2552ef23b337 100644 --- a/be/src/service/doris_main.cpp +++ b/be/src/service/doris_main.cpp @@ -536,7 +536,7 @@ int main(int argc, char** argv) { // 5. arrow flight service std::shared_ptr flight_server = std::move(doris::flight::FlightSqlServer::create()).ValueOrDie(); - status = flight_server->init(doris::config::arrow_flight_port); + status = flight_server->init(doris::config::arrow_flight_sql_port); // 6. start daemon thread to do clean or gc jobs doris::Daemon daemon; diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index 525e9ea0248470..5367135c134ada 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -96,6 +96,7 @@ #include "runtime/thread_context.h" #include "runtime/types.h" #include "service/point_query_executor.h" +#include "util/arrow/row_batch.h" #include "util/async_io.h" #include "util/brpc_client_cache.h" #include "util/doris_metrics.h" @@ -704,6 +705,40 @@ void PInternalServiceImpl::fetch_table_schema(google::protobuf::RpcController* c } } +void PInternalServiceImpl::fetch_arrow_flight_schema(google::protobuf::RpcController* controller, + const PFetchArrowFlightSchemaRequest* request, + PFetchArrowFlightSchemaResult* result, + google::protobuf::Closure* done) { + bool ret = _light_work_pool.try_offer([request, result, done]() { + brpc::ClosureGuard closure_guard(done); + RowDescriptor row_desc = ExecEnv::GetInstance()->result_mgr()->find_row_descriptor( + UniqueId(request->finst_id()).to_thrift()); + if (row_desc.equals(RowDescriptor())) { + auto st = Status::NotFound("not found row descriptor"); + st.to_protobuf(result->mutable_status()); + return; + } + + std::shared_ptr schema; + auto st = convert_to_arrow_schema(row_desc, &schema); + if (UNLIKELY(!st.ok())) { + st.to_protobuf(result->mutable_status()); + return; + } + + std::string schema_str; + st = serialize_arrow_schema(row_desc, &schema, &schema_str); + if (st.ok()) { + result->set_schema(std::move(schema_str)); + } + st.to_protobuf(result->mutable_status()); + }); + if (!ret) { + offer_failed(result, done, _heavy_work_pool); + return; + } +} + Status PInternalServiceImpl::_tablet_fetch_data(const PTabletKeyLookupRequest* request, PTabletKeyLookupResponse* response) { PointQueryExecutor lookup_util; diff --git a/be/src/service/internal_service.h b/be/src/service/internal_service.h index e7a5914274cee7..db0ee07581ef02 100644 --- a/be/src/service/internal_service.h +++ b/be/src/service/internal_service.h @@ -82,6 +82,11 @@ class PInternalServiceImpl : public PBackendService { PFetchTableSchemaResult* result, google::protobuf::Closure* done) override; + void fetch_arrow_flight_schema(google::protobuf::RpcController* controller, + const PFetchArrowFlightSchemaRequest* request, + PFetchArrowFlightSchemaResult* result, + google::protobuf::Closure* done) override; + void tablet_writer_open(google::protobuf::RpcController* controller, const PTabletWriterOpenRequest* request, PTabletWriterOpenResult* response, diff --git a/be/src/util/arrow/row_batch.cpp b/be/src/util/arrow/row_batch.cpp index 78fe346be6cb1b..b60034696acd4f 100644 --- a/be/src/util/arrow/row_batch.cpp +++ b/be/src/util/arrow/row_batch.cpp @@ -37,6 +37,8 @@ #include "runtime/define_primitive_type.h" #include "runtime/descriptors.h" #include "runtime/types.h" +#include "util/arrow/block_convertor.h" +#include "vec/core/block.h" namespace doris { @@ -188,4 +190,16 @@ Status serialize_record_batch(const arrow::RecordBatch& record_batch, std::strin return Status::OK(); } +Status serialize_arrow_schema(RowDescriptor row_desc, std::shared_ptr* schema, + std::string* result) { + std::vector slots; + for (auto tuple_desc : row_desc.tuple_descriptors()) { + slots.insert(slots.end(), tuple_desc->slots().begin(), tuple_desc->slots().end()); + } + auto block = vectorized::Block(slots, 0); + std::shared_ptr batch; + RETURN_IF_ERROR(convert_to_arrow_batch(block, *schema, arrow::default_memory_pool(), &batch)); + return serialize_record_batch(*batch, result); +} + } // namespace doris diff --git a/be/src/util/arrow/row_batch.h b/be/src/util/arrow/row_batch.h index 8946502f839cd3..b5e9d8d3c3781c 100644 --- a/be/src/util/arrow/row_batch.h +++ b/be/src/util/arrow/row_batch.h @@ -43,4 +43,7 @@ Status convert_to_arrow_schema(const RowDescriptor& row_desc, Status serialize_record_batch(const arrow::RecordBatch& record_batch, std::string* result); +Status serialize_arrow_schema(RowDescriptor row_desc, std::shared_ptr* schema, + std::string* result); + } // namespace doris diff --git a/be/src/util/timezone_utils.cpp b/be/src/util/timezone_utils.cpp index 14dd8b7e53fadd..be8c282b3a6483 100644 --- a/be/src/util/timezone_utils.cpp +++ b/be/src/util/timezone_utils.cpp @@ -43,6 +43,12 @@ std::unordered_map TimezoneUtils::timezone_names_map_; bool TimezoneUtils::inited_ = false; const std::string TimezoneUtils::default_time_zone = "+08:00"; +static const char* tzdir = "/usr/share/zoneinfo"; // default value, may change by TZDIR env var + +void TimezoneUtils::clear_timezone_names() { + timezone_names_map_.clear(); + inited_ = false; +} void TimezoneUtils::load_timezone_names() { if (inited_) { @@ -51,7 +57,6 @@ void TimezoneUtils::load_timezone_names() { inited_ = true; std::string path; - const char* tzdir = "/usr/share/zoneinfo"; char* tzdir_env = std::getenv("TZDIR"); if (tzdir_env && *tzdir_env) { tzdir = tzdir_env; @@ -210,7 +215,6 @@ void TimezoneUtils::load_timezones_to_cache(vectorized::ZoneList& cache_list) { cache_list["CST"] = cctz::fixed_time_zone(cctz::seconds(8 * 3600)); std::string base_str; - const char* tzdir = "/usr/share/zoneinfo"; // default // try get from System char* tzdir_env = std::getenv("TZDIR"); if (tzdir_env && *tzdir_env) { @@ -221,6 +225,11 @@ void TimezoneUtils::load_timezones_to_cache(vectorized::ZoneList& cache_list) { base_str += '/'; const auto root_path = std::filesystem::path {base_str}; + if (!std::filesystem::exists(root_path)) { + LOG_WARNING("Cannot find system tzfile. Abandon to preload timezone cache."); + return; + } + std::set ignore_paths = {"posix", "right"}; // duplications for (std::filesystem::recursive_directory_iterator it {base_str}; it != end(it); it++) { @@ -295,11 +304,11 @@ bool TimezoneUtils::find_cctz_time_zone(const std::string& timezone, cctz::time_ tz_parsed = true; } else { auto it = timezone_names_map_.find(timezone_lower); - if (it == timezone_names_map_.end()) { - VLOG_DEBUG << "Illegal timezone " << timezone_lower; - return false; + if (it != timezone_names_map_.end()) { + tz_parsed = cctz::load_time_zone(it->second, &ctz); + } else { + tz_parsed = cctz::load_time_zone(timezone, &ctz); } - tz_parsed = cctz::load_time_zone(it->second, &ctz); } if (tz_parsed) { if (!have_both) { // GMT only diff --git a/be/src/util/timezone_utils.h b/be/src/util/timezone_utils.h index 0f3a6dcc38a0e1..55f7eace20f4d1 100644 --- a/be/src/util/timezone_utils.h +++ b/be/src/util/timezone_utils.h @@ -37,8 +37,12 @@ class TimezoneUtils { public: static void load_timezone_names(); static void load_timezones_to_cache(vectorized::ZoneList& cache_list); + // we support to parse lower_case timezone name iff execution environment has timezone file static bool find_cctz_time_zone(const std::string& timezone, cctz::time_zone& ctz); + // for ut only + static void clear_timezone_names(); + static const std::string default_time_zone; private: diff --git a/be/src/vec/exec/format/csv/csv_reader.cpp b/be/src/vec/exec/format/csv/csv_reader.cpp index eeb3aac416bc71..93769a97c9c90e 100644 --- a/be/src/vec/exec/format/csv/csv_reader.cpp +++ b/be/src/vec/exec/format/csv/csv_reader.cpp @@ -139,9 +139,27 @@ void PlainCsvTextFieldSplitter::_split_field_multi_char(const Slice& line, if (j == value_sep_len - 1) { curpos = i - value_sep_len + 1; - process_value_func(line.data, start, curpos - start, trimming_char, splitted_values); + /* + * column_separator : "xx" + * data.csv : data1xxxxdata2 + * + * Parse incorrectly: + * data1[xx]xxdata2 + * data1x[xx]xdata2 + * data1xx[xx]data2 + * The string "xxxx" is parsed into three "xx" delimiters. + * + * Parse correctly: + * data1[xx]xxdata2 + * data1xx[xx]data2 + */ + + if (curpos >= start) { + process_value_func(line.data, start, curpos - start, trimming_char, + splitted_values); + start = i + 1; + } - start = i + 1; j = next[j]; } } diff --git a/be/src/vec/exec/scan/pip_scanner_context.h b/be/src/vec/exec/scan/pip_scanner_context.h index 2022ecb29e6606..6360f104c6347d 100644 --- a/be/src/vec/exec/scan/pip_scanner_context.h +++ b/be/src/vec/exec/scan/pip_scanner_context.h @@ -142,13 +142,13 @@ class PipScannerContext : public vectorized::ScannerContext { for (int j = i; j < block_size; j += queue_size) { _blocks_queues[queue].emplace_back(std::move(blocks[j])); } + if (_data_dependency) { + _data_dependency->set_ready_for_read(); + } } _next_queue_to_feed = queue + 1 < queue_size ? queue + 1 : 0; } } - if (_data_dependency) { - _data_dependency->set_ready_for_read(); - } _current_used_bytes += local_bytes; } diff --git a/be/src/vec/exec/vjdbc_connector.cpp b/be/src/vec/exec/vjdbc_connector.cpp index 59d8b4b1a1311b..65ebc428824e35 100644 --- a/be/src/vec/exec/vjdbc_connector.cpp +++ b/be/src/vec/exec/vjdbc_connector.cpp @@ -90,7 +90,7 @@ JdbcConnector::~JdbcConnector() { #define DELETE_BASIC_JAVA_CLAZZ_REF(CPP_TYPE) env->DeleteGlobalRef(_executor_##CPP_TYPE##_clazz); -Status JdbcConnector::close() { +Status JdbcConnector::close(Status) { SCOPED_RAW_TIMER(&_jdbc_statistic._connector_close_timer); _closed = true; if (!_is_open) { diff --git a/be/src/vec/exec/vjdbc_connector.h b/be/src/vec/exec/vjdbc_connector.h index 50f7d669ca8b0d..d1587af87f5864 100644 --- a/be/src/vec/exec/vjdbc_connector.h +++ b/be/src/vec/exec/vjdbc_connector.h @@ -106,7 +106,7 @@ class JdbcConnector : public TableConnector { JdbcStatistic& get_jdbc_statistic() { return _jdbc_statistic; } - Status close() override; + Status close(Status s = Status::OK()) override; protected: JdbcConnectorParam _conn_param; diff --git a/be/src/vec/functions/array/function_array_map.h b/be/src/vec/functions/array/function_array_map.h index be9cd1be3d1929..a586e262cb20ca 100644 --- a/be/src/vec/functions/array/function_array_map.h +++ b/be/src/vec/functions/array/function_array_map.h @@ -27,13 +27,33 @@ #include "vec/functions/array/function_array_utils.h" #include "vec/functions/function_helpers.h" +#define FILL_MAP_DATA_INTO_DEFAULT_COLUMN() \ + ++dst_off; \ + auto& dst_data = static_cast(*dst.nested_col).get_data(); \ + dst_data.push_back(entry.get_first()); \ + if (dst.nested_nullmap_data) { \ + dst.nested_nullmap_data->push_back(0); \ + } + +#define FILL_MAP_DATA_INTO_STRING_COLUMN() \ + auto& dst_col = static_cast(*dst.nested_col); \ + StringRef key = entry.get_first(); \ + ++dst_off; \ + dst_col.insert_data(key.data, key.size); \ + if (dst.nested_nullmap_data) { \ + dst.nested_nullmap_data->push_back(0); \ + } + namespace doris::vectorized { -enum class MapOperation { INTERSECT }; +enum class MapOperation { INTERSECT, UNION }; template struct IntersectAction; +template +struct UnionAction; + template struct MapActionImpl; @@ -42,6 +62,11 @@ struct MapActionImpl { using Action = IntersectAction; }; +template +struct MapActionImpl { + using Action = UnionAction; +}; + template struct OpenMapImpl { using Element = typename ColumnType::value_type; @@ -62,7 +87,7 @@ struct OpenMapImpl { std::vector& col_const, int start_row, int end_row) { size_t dst_off = 0; for (int row = start_row; row < end_row; ++row) { - map.clear(); + reset(); for (int i = 0; i < params.size(); ++i) { action.apply(map, i, index_check_const(row, col_const[i]), params[i]); } @@ -76,13 +101,13 @@ struct OpenMapImpl { } // make map result to dst for (const auto& entry : map) { - if (entry.get_mapped() == params.size()) { - ++dst_off; - auto& dst_data = static_cast(*dst.nested_col).get_data(); - dst_data.push_back(entry.get_first()); - if (dst.nested_nullmap_data) { - dst.nested_nullmap_data->push_back(0); + if constexpr (operation == MapOperation::INTERSECT) { + if (entry.get_mapped() == params.size()) { + FILL_MAP_DATA_INTO_DEFAULT_COLUMN() } + } else if constexpr (operation == MapOperation::UNION) { + // union in map all key + FILL_MAP_DATA_INTO_DEFAULT_COLUMN() } } dst.offsets_ptr->push_back(dst_off); @@ -107,7 +132,7 @@ struct OpenMapImpl { std::vector& col_const, int start_row, int end_row) { size_t dst_off = 0; for (int row = start_row; row < end_row; ++row) { - map.clear(); + reset(); for (int i = 0; i < params.size(); ++i) { action.apply(map, i, index_check_const(row, col_const[i]), params[i]); } @@ -121,14 +146,12 @@ struct OpenMapImpl { } // make map result to dst for (const auto& entry : map) { - if (entry.get_mapped() == params.size()) { - auto& dst_col = static_cast(*dst.nested_col); - StringRef key = entry.get_first(); - ++dst_off; - dst_col.insert_data(key.data, key.size); - if (dst.nested_nullmap_data) { - dst.nested_nullmap_data->push_back(0); + if constexpr (operation == MapOperation::INTERSECT) { + if (entry.get_mapped() == params.size()) { + FILL_MAP_DATA_INTO_STRING_COLUMN() } + } else if constexpr (operation == MapOperation::UNION) { + FILL_MAP_DATA_INTO_STRING_COLUMN() } } dst.offsets_ptr->push_back(dst_off); diff --git a/be/src/vec/functions/array/function_array_set.h b/be/src/vec/functions/array/function_array_set.h index 1012fd055e6f4e..41bf53f921f3fb 100644 --- a/be/src/vec/functions/array/function_array_set.h +++ b/be/src/vec/functions/array/function_array_set.h @@ -39,11 +39,6 @@ struct ExceptAction; template struct ActionImpl; -template -struct ActionImpl { - using Action = UnionAction; -}; - template struct ActionImpl { using Action = ExceptAction; diff --git a/be/src/vec/functions/array/function_array_union.cpp b/be/src/vec/functions/array/function_array_union.cpp index 68c9c2043fec2a..75f7ec211b00c1 100644 --- a/be/src/vec/functions/array/function_array_union.cpp +++ b/be/src/vec/functions/array/function_array_union.cpp @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +#include "function_array_map.h" #include "vec/columns/column_const.h" #include "vec/common/assert_cast.h" #include "vec/common/string_ref.h" @@ -31,41 +32,43 @@ struct NameArrayUnion { static constexpr auto name = "array_union"; }; -template +template struct UnionAction { - // True if set has null element - bool null_flag = false; + // True if current has null element + bool current_null_flag = false; // True if result_set has null element bool result_null_flag = false; - // True if it should execute the left array first. - static constexpr auto execute_left_column_first = true; // Handle Null element. - // Return true means this null element should put into result column. - template - bool apply_null() { - if (!null_flag) { - null_flag = true; - return true; - } - return false; - } + bool apply_null() { return result_null_flag; } // Handle Non-Null element. - // Return ture means this Non-Null element should put into result column. - template - bool apply(Set& set, Set& result_set, const Element& elem) { - if (!set.find(elem)) { - set.insert(elem); - return true; + void apply(Map& map, size_t arg_idx, size_t row_idx, const ColumnArrayExecutionData& param) { + current_null_flag = false; + size_t start_off = (*param.offsets_ptr)[row_idx - 1]; + size_t end_off = (*param.offsets_ptr)[row_idx]; + for (size_t off = start_off; off < end_off; ++off) { + if (param.nested_nullmap_data && param.nested_nullmap_data[off]) { + current_null_flag = true; + } else { + if constexpr (std::is_same_v) { + map[param.nested_col->get_data_at(off)]; + } else { + auto& data_col = static_cast(*param.nested_col); + map[data_col.get_element(off)]; + } + } } - return false; + result_null_flag = result_null_flag || current_null_flag; } - void reset() { null_flag = false; } + void reset() { + current_null_flag = false; + result_null_flag = false; + } }; -using FunctionArrayUnion = FunctionArrayBinary, NameArrayUnion>; +using FunctionArrayUnion = FunctionArrayNary, NameArrayUnion>; void register_function_array_union(SimpleFunctionFactory& factory) { factory.register_function(); diff --git a/be/src/vec/functions/function_convert_tz.h b/be/src/vec/functions/function_convert_tz.h index 8ff3505acaa317..d2db44c1178f67 100644 --- a/be/src/vec/functions/function_convert_tz.h +++ b/be/src/vec/functions/function_convert_tz.h @@ -139,7 +139,7 @@ struct ConvertTZImpl { std::unique_lock lock_(cache_lock); //TODO: the lock upgrade could be done in find_... function only when we push value into the hashmap if (!TimezoneUtils::find_cctz_time_zone(from_tz, time_zone_cache[from_tz])) { - time_zone_cache.erase(to_tz); + time_zone_cache.erase(from_tz); result_null_map[index_now] = true; result_column->insert_default(); return; diff --git a/be/src/vec/sink/async_writer_sink.h b/be/src/vec/sink/async_writer_sink.h index 8eb177ce8eedfe..5fd03762a5580a 100644 --- a/be/src/vec/sink/async_writer_sink.h +++ b/be/src/vec/sink/async_writer_sink.h @@ -94,14 +94,14 @@ class AsyncWriterSink : public DataSink { if (exec_status.ok() && !state->is_cancelled()) { RETURN_IF_ERROR(_writer->commit_trans()); } - RETURN_IF_ERROR(_writer->close()); + RETURN_IF_ERROR(_writer->close(exec_status)); } return DataSink::close(state, exec_status); } Status try_close(RuntimeState* state, Status exec_status) override { if (state->is_cancelled() || !exec_status.ok()) { - _writer->force_close(); + _writer->force_close(!exec_status.ok() ? exec_status : Status::Cancelled("Cancelled")); } return Status::OK(); } diff --git a/be/src/vec/sink/autoinc_buffer.cpp b/be/src/vec/sink/autoinc_buffer.cpp index 675f8ee2925885..58527f38f70521 100644 --- a/be/src/vec/sink/autoinc_buffer.cpp +++ b/be/src/vec/sink/autoinc_buffer.cpp @@ -25,8 +25,7 @@ #include "util/thrift_rpc_helper.h" #include "vec/sink/vtablet_block_convertor.h" -namespace doris { -namespace stream_load { +namespace doris::vectorized { FetchAutoIncIDExecutor::FetchAutoIncIDExecutor() { ThreadPoolBuilder("AsyncFetchAutoIncIDExecutor") @@ -125,5 +124,4 @@ void AutoIncIDBuffer::_prefetch_ids(size_t length) { }); } -} // namespace stream_load -} // namespace doris \ No newline at end of file +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/sink/autoinc_buffer.h b/be/src/vec/sink/autoinc_buffer.h index 870418c0de10e1..3443386c82bfa9 100644 --- a/be/src/vec/sink/autoinc_buffer.h +++ b/be/src/vec/sink/autoinc_buffer.h @@ -22,8 +22,7 @@ #include "common/status.h" #include "util/threadpool.h" -namespace doris { -namespace stream_load { +namespace doris::vectorized { class VOlapTableSink; class OlapTableBlockConvertor; @@ -126,5 +125,4 @@ class GlobalAutoIncBuffers { std::mutex _mutex; }; -} // namespace stream_load -} // namespace doris \ No newline at end of file +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/sink/delta_writer_v2_pool.cpp b/be/src/vec/sink/delta_writer_v2_pool.cpp index c1066174a8cdc2..0f86efd8bcdc8e 100644 --- a/be/src/vec/sink/delta_writer_v2_pool.cpp +++ b/be/src/vec/sink/delta_writer_v2_pool.cpp @@ -22,7 +22,7 @@ namespace doris { class TExpr; -namespace stream_load { +namespace vectorized { DeltaWriterV2Map::DeltaWriterV2Map(UniqueId load_id) : _load_id(load_id), _use_cnt(1) {} @@ -83,5 +83,5 @@ std::shared_ptr DeltaWriterV2Pool::get_or_create(PUniqueId loa return map; } -} // namespace stream_load +} // namespace vectorized } // namespace doris diff --git a/be/src/vec/sink/delta_writer_v2_pool.h b/be/src/vec/sink/delta_writer_v2_pool.h index d0328d7d2b5b08..7654c2ca7eddf8 100644 --- a/be/src/vec/sink/delta_writer_v2_pool.h +++ b/be/src/vec/sink/delta_writer_v2_pool.h @@ -55,7 +55,7 @@ namespace doris { class DeltaWriterV2; -namespace stream_load { +namespace vectorized { class DeltaWriterV2Map { public: @@ -107,5 +107,5 @@ class DeltaWriterV2Pool { std::unordered_map> _pool; }; -} // namespace stream_load +} // namespace vectorized } // namespace doris \ No newline at end of file diff --git a/be/src/vec/sink/group_commit_vtablet_sink.cpp b/be/src/vec/sink/group_commit_vtablet_sink.cpp deleted file mode 100644 index 3bf51993fde41c..00000000000000 --- a/be/src/vec/sink/group_commit_vtablet_sink.cpp +++ /dev/null @@ -1,40 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "vec/sink/group_commit_vtablet_sink.h" - -#include "vec/core/future_block.h" - -namespace doris { - -namespace stream_load { - -GroupCommitVOlapTableSink::GroupCommitVOlapTableSink(ObjectPool* pool, - const RowDescriptor& row_desc, - const std::vector& texprs, - Status* status) - : VOlapTableSink(pool, row_desc, texprs, status) {} - -void GroupCommitVOlapTableSink::handle_block(vectorized::Block* input_block, int64_t rows, - int64_t filter_rows) { - auto* future_block = dynamic_cast(input_block); - std::unique_lock l(*(future_block->lock)); - future_block->set_result(Status::OK(), rows, rows - filter_rows); - future_block->cv->notify_all(); -} -} // namespace stream_load -} // namespace doris diff --git a/be/src/vec/sink/group_commit_vtablet_sink.h b/be/src/vec/sink/group_commit_vtablet_sink.h deleted file mode 100644 index 1c9e64355b6157..00000000000000 --- a/be/src/vec/sink/group_commit_vtablet_sink.h +++ /dev/null @@ -1,34 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once -#include "vtablet_sink.h" - -namespace doris { - -namespace stream_load { - -class GroupCommitVOlapTableSink : public VOlapTableSink { -public: - GroupCommitVOlapTableSink(ObjectPool* pool, const RowDescriptor& row_desc, - const std::vector& texprs, Status* status); - - void handle_block(vectorized::Block* input_block, int64_t rows, int64_t filter_rows) override; -}; - -} // namespace stream_load -} // namespace doris diff --git a/be/src/vec/sink/varrow_flight_result_writer.cpp b/be/src/vec/sink/varrow_flight_result_writer.cpp index 88574ca99728dd..4a71e10df426a6 100644 --- a/be/src/vec/sink/varrow_flight_result_writer.cpp +++ b/be/src/vec/sink/varrow_flight_result_writer.cpp @@ -100,7 +100,7 @@ bool VArrowFlightResultWriter::can_sink() { return _sinker->can_sink(); } -Status VArrowFlightResultWriter::close() { +Status VArrowFlightResultWriter::close(Status) { COUNTER_SET(_sent_rows_counter, _written_rows); COUNTER_UPDATE(_bytes_sent_counter, _bytes_sent); return Status::OK(); diff --git a/be/src/vec/sink/varrow_flight_result_writer.h b/be/src/vec/sink/varrow_flight_result_writer.h index b95fdeae722f95..7aa8ec6824a7e6 100644 --- a/be/src/vec/sink/varrow_flight_result_writer.h +++ b/be/src/vec/sink/varrow_flight_result_writer.h @@ -47,7 +47,7 @@ class VArrowFlightResultWriter final : public ResultWriter { bool can_sink() override; - Status close() override; + Status close(Status) override; private: void _init_profile(); diff --git a/be/src/vec/sink/vmysql_result_writer.cpp b/be/src/vec/sink/vmysql_result_writer.cpp index ee4587879943d5..208f0a2fddfd49 100644 --- a/be/src/vec/sink/vmysql_result_writer.cpp +++ b/be/src/vec/sink/vmysql_result_writer.cpp @@ -187,7 +187,7 @@ bool VMysqlResultWriter::can_sink() { } template -Status VMysqlResultWriter::close() { +Status VMysqlResultWriter::close(Status) { COUNTER_SET(_sent_rows_counter, _written_rows); COUNTER_UPDATE(_bytes_sent_counter, _bytes_sent); return Status::OK(); diff --git a/be/src/vec/sink/vmysql_result_writer.h b/be/src/vec/sink/vmysql_result_writer.h index b1954f2b01904d..f86b62b7e2a153 100644 --- a/be/src/vec/sink/vmysql_result_writer.h +++ b/be/src/vec/sink/vmysql_result_writer.h @@ -51,7 +51,7 @@ class VMysqlResultWriter final : public ResultWriter { bool can_sink() override; - Status close() override; + Status close(Status status) override; const ResultList& results() { return _results; } diff --git a/be/src/vec/sink/vresult_file_sink.cpp b/be/src/vec/sink/vresult_file_sink.cpp index 66d64140814485..82f4e7e805abee 100644 --- a/be/src/vec/sink/vresult_file_sink.cpp +++ b/be/src/vec/sink/vresult_file_sink.cpp @@ -69,7 +69,6 @@ Status VResultFileSink::init(const TDataSink& tsink) { CHECK(sink.__isset.storage_backend_type); _storage_type = sink.storage_backend_type; - _name = "VResultFileSink"; //for impl csv_with_name and csv_with_names_and_types _header_type = sink.header_type; _header = sink.header; diff --git a/be/src/vec/sink/vtablet_block_convertor.cpp b/be/src/vec/sink/vtablet_block_convertor.cpp index 92536697c9375e..d05a1a9257af1e 100644 --- a/be/src/vec/sink/vtablet_block_convertor.cpp +++ b/be/src/vec/sink/vtablet_block_convertor.cpp @@ -52,13 +52,12 @@ #include "vec/exprs/vexpr.h" #include "vec/exprs/vexpr_context.h" -namespace doris { -namespace stream_load { +namespace doris::vectorized { Status OlapTableBlockConvertor::validate_and_convert_block( RuntimeState* state, vectorized::Block* input_block, std::shared_ptr& block, vectorized::VExprContextSPtrs output_vexpr_ctxs, - size_t rows, bool eos, bool& has_filtered_rows) { + size_t rows, bool& has_filtered_rows) { DCHECK(input_block->rows() > 0); block = vectorized::Block::create_shared(input_block->get_columns_with_type_and_name()); @@ -70,7 +69,7 @@ Status OlapTableBlockConvertor::validate_and_convert_block( // fill the valus for auto-increment columns if (_auto_inc_col_idx.has_value()) { - RETURN_IF_ERROR(_fill_auto_inc_cols(block.get(), rows, eos)); + RETURN_IF_ERROR(_fill_auto_inc_cols(block.get(), rows)); } int64_t filtered_rows = 0; @@ -450,8 +449,7 @@ void OlapTableBlockConvertor::_convert_to_dest_desc_block(doris::vectorized::Blo } } -Status OlapTableBlockConvertor::_fill_auto_inc_cols(vectorized::Block* block, size_t rows, - bool eos) { +Status OlapTableBlockConvertor::_fill_auto_inc_cols(vectorized::Block* block, size_t rows) { size_t idx = _auto_inc_col_idx.value(); SlotDescriptor* slot = _output_tuple_desc->slots()[idx]; DCHECK(slot->type().type == PrimitiveType::TYPE_BIGINT); @@ -513,5 +511,4 @@ Status OlapTableBlockConvertor::_fill_auto_inc_cols(vectorized::Block* block, si return Status::OK(); } -} // namespace stream_load -} // namespace doris \ No newline at end of file +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/sink/vtablet_block_convertor.h b/be/src/vec/sink/vtablet_block_convertor.h index bfc7b3b5d969de..bf5148cf0a72bd 100644 --- a/be/src/vec/sink/vtablet_block_convertor.h +++ b/be/src/vec/sink/vtablet_block_convertor.h @@ -35,8 +35,7 @@ #include "vec/exprs/vexpr_fwd.h" #include "vec/sink/autoinc_buffer.h" -namespace doris { -namespace stream_load { +namespace doris::vectorized { class OlapTableBlockConvertor { public: @@ -46,7 +45,7 @@ class OlapTableBlockConvertor { Status validate_and_convert_block(RuntimeState* state, vectorized::Block* input_block, std::shared_ptr& block, vectorized::VExprContextSPtrs output_vexpr_ctxs, size_t rows, - bool eos, bool& has_filtered_rows); + bool& has_filtered_rows); const Bitmap& filter_bitmap() { return _filter_bitmap; } @@ -81,7 +80,7 @@ class OlapTableBlockConvertor { // so here need to do the convert operation void _convert_to_dest_desc_block(vectorized::Block* block); - Status _fill_auto_inc_cols(vectorized::Block* block, size_t rows, bool eos); + Status _fill_auto_inc_cols(vectorized::Block* block, size_t rows); TupleDescriptor* _output_tuple_desc = nullptr; @@ -106,5 +105,4 @@ class OlapTableBlockConvertor { AutoIncIDAllocator _auto_inc_id_allocator; }; -} // namespace stream_load -} // namespace doris \ No newline at end of file +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/sink/vtablet_finder.cpp b/be/src/vec/sink/vtablet_finder.cpp index e432c864974099..5d030a7ffab88a 100644 --- a/be/src/vec/sink/vtablet_finder.cpp +++ b/be/src/vec/sink/vtablet_finder.cpp @@ -40,10 +40,9 @@ #include "vec/data_types/data_type.h" #include "vec/functions/simple_function_factory.h" -namespace doris { -namespace stream_load { +namespace doris::vectorized { -Status OlapTabletFinder::find_tablet(RuntimeState* state, vectorized::Block* block, int row_index, +Status OlapTabletFinder::find_tablet(RuntimeState* state, Block* block, int row_index, const VOlapTablePartition** partition, uint32_t& tablet_index, bool& stop_processing, bool& is_continue, bool* missing_partition) { @@ -99,5 +98,4 @@ Status OlapTabletFinder::find_tablet(RuntimeState* state, vectorized::Block* blo return status; } -} // namespace stream_load -} // namespace doris \ No newline at end of file +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/sink/vtablet_finder.h b/be/src/vec/sink/vtablet_finder.h index 9fe64944300f35..2258e15de08161 100644 --- a/be/src/vec/sink/vtablet_finder.h +++ b/be/src/vec/sink/vtablet_finder.h @@ -23,8 +23,7 @@ #include "exec/tablet_info.h" #include "vec/core/block.h" -namespace doris { -namespace stream_load { +namespace doris::vectorized { class OlapTabletFinder { public: @@ -73,5 +72,4 @@ class OlapTabletFinder { int64_t _num_immutable_partition_filtered_rows = 0; }; -} // namespace stream_load -} // namespace doris \ No newline at end of file +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/sink/vtablet_sink.cpp b/be/src/vec/sink/vtablet_sink.cpp index c6b9dbe673fc3a..39f4bd6ef93411 100644 --- a/be/src/vec/sink/vtablet_sink.cpp +++ b/be/src/vec/sink/vtablet_sink.cpp @@ -124,1829 +124,27 @@ namespace doris { class TExpr; -namespace stream_load { - -// an IndexChannel is related to specific table and its rollup and mv -class IndexChannel { -public: - IndexChannel(VOlapTableSink* parent, int64_t index_id, - const vectorized::VExprContextSPtr& where_clause) - : _parent(parent), _index_id(index_id), _where_clause(where_clause) { - _index_channel_tracker = - std::make_unique("IndexChannel:indexID=" + std::to_string(_index_id)); - } - ~IndexChannel() = default; - - // allow to init multi times, for incremental open more tablets for one index(table) - Status init(RuntimeState* state, const std::vector& tablets); - - void for_each_node_channel( - const std::function&)>& func) { - for (auto& it : _node_channels) { - func(it.second); - } - } - - void mark_as_failed(const VNodeChannel* node_channel, const std::string& err, - int64_t tablet_id = -1); - Status check_intolerable_failure(); - - // set error tablet info in runtime state, so that it can be returned to FE. - void set_error_tablet_in_state(RuntimeState* state); - - size_t num_node_channels() const { return _node_channels.size(); } - - size_t get_pending_bytes() const { - size_t mem_consumption = 0; - for (auto& kv : _node_channels) { - mem_consumption += kv.second->get_pending_bytes(); - } - return mem_consumption; - } - - void set_tablets_received_rows( - const std::vector>& tablets_received_rows, int64_t node_id); - void set_tablets_filtered_rows( - const std::vector>& tablets_filtered_rows, int64_t node_id); - int64_t num_rows_filtered() { - DCHECK(!_tablets_filtered_rows.empty()); - // the Unique table has no roll up or materilized view - // we just add up filtered rows from all partitions - return std::accumulate(_tablets_filtered_rows.cbegin(), _tablets_filtered_rows.cend(), 0, - [](int64_t sum, const auto& a) { return sum + a.second[0].second; }); - } - - // check whether the rows num written by different replicas is consistent - Status check_tablet_received_rows_consistency(); - // check whether the rows num filtered by different replicas is consistent - Status check_tablet_filtered_rows_consistency(); - - vectorized::VExprContextSPtr get_where_clause() { return _where_clause; } - -private: - friend class VNodeChannel; - friend class VOlapTableSink; - - VOlapTableSink* _parent; - int64_t _index_id; - vectorized::VExprContextSPtr _where_clause; - - // from backend channel to tablet_id - // ATTN: must be placed before `_node_channels` and `_channels_by_tablet`. - // Because the destruct order of objects is opposite to the creation order. - // So NodeChannel will be destructured first. - // And the destructor function of NodeChannel waits for all RPCs to finish. - // This ensures that it is safe to use `_tablets_by_channel` in the callback function for the end of the RPC. - std::unordered_map> _tablets_by_channel; - // BeId -> channel - std::unordered_map> _node_channels; - // from tablet_id to backend channel - std::unordered_map>> _channels_by_tablet; - - // lock to protect _failed_channels and _failed_channels_msgs - mutable doris::SpinLock _fail_lock; - // key is tablet_id, value is a set of failed node id - std::unordered_map> _failed_channels; - // key is tablet_id, value is error message - std::unordered_map _failed_channels_msgs; - Status _intolerable_failure_status = Status::OK(); - - std::unique_ptr _index_channel_tracker; - // rows num received by DeltaWriter per tablet, tablet_id -> - // used to verify whether the rows num received by different replicas is consistent - std::map>> _tablets_received_rows; - // rows num filtered by DeltaWriter per tablet, tablet_id -> - // used to verify whether the rows num filtered by different replicas is consistent - std::map>> _tablets_filtered_rows; -}; - -Status IndexChannel::init(RuntimeState* state, const std::vector& tablets) { - SCOPED_CONSUME_MEM_TRACKER(_index_channel_tracker.get()); - for (auto& tablet : tablets) { - // First find the location BEs of this tablet - auto tablet_locations = _parent->_location->find_tablet(tablet.tablet_id); - if (tablet_locations == nullptr) { - return Status::InternalError("unknown tablet, tablet_id={}", tablet.tablet_id); - } - std::vector> channels; - // For tablet, deal with its' all replica (in some node). - for (auto& replica_node_id : tablet_locations->node_ids) { - std::shared_ptr channel; - auto it = _node_channels.find(replica_node_id); - // when we prepare for TableSink or incremental open tablet, we need init - if (it == _node_channels.end()) { - // NodeChannel is not added to the _parent->_pool. - // Because the deconstruction of NodeChannel may take a long time to wait rpc finish. - // but the ObjectPool will hold a spin lock to delete objects. - channel = std::make_shared(_parent, this, replica_node_id); - _node_channels.emplace(replica_node_id, channel); - } else { - channel = it->second; - } - channel->add_tablet(tablet); - if (_parent->_write_single_replica) { - auto slave_location = _parent->_slave_location->find_tablet(tablet.tablet_id); - if (slave_location != nullptr) { - channel->add_slave_tablet_nodes(tablet.tablet_id, slave_location->node_ids); - } - } - channels.push_back(channel); - _tablets_by_channel[replica_node_id].insert(tablet.tablet_id); - } - _channels_by_tablet.emplace(tablet.tablet_id, std::move(channels)); - } - for (auto& it : _node_channels) { - RETURN_IF_ERROR(it.second->init(state)); - } - if (_where_clause != nullptr) { - RETURN_IF_ERROR(_where_clause->prepare(state, *_parent->_output_row_desc)); - RETURN_IF_ERROR(_where_clause->open(state)); - } - return Status::OK(); -} - -void IndexChannel::mark_as_failed(const VNodeChannel* node_channel, const std::string& err, - int64_t tablet_id) { - DCHECK(node_channel != nullptr); - LOG(INFO) << "mark node_id:" << node_channel->channel_info() << " tablet_id: " << tablet_id - << " as failed, err: " << err; - auto node_id = node_channel->node_id(); - const auto& it = _tablets_by_channel.find(node_id); - if (it == _tablets_by_channel.end()) { - return; - } - - { - std::lock_guard l(_fail_lock); - if (tablet_id == -1) { - for (const auto the_tablet_id : it->second) { - _failed_channels[the_tablet_id].insert(node_id); - _failed_channels_msgs.emplace(the_tablet_id, - err + ", host: " + node_channel->host()); - if (_failed_channels[the_tablet_id].size() >= ((_parent->_num_replicas + 1) / 2)) { - _intolerable_failure_status = - Status::InternalError(_failed_channels_msgs[the_tablet_id]); - } - } - } else { - _failed_channels[tablet_id].insert(node_id); - _failed_channels_msgs.emplace(tablet_id, err + ", host: " + node_channel->host()); - if (_failed_channels[tablet_id].size() >= ((_parent->_num_replicas + 1) / 2)) { - _intolerable_failure_status = - Status::InternalError(_failed_channels_msgs[tablet_id]); - } - } - } -} - -Status IndexChannel::check_intolerable_failure() { - std::lock_guard l(_fail_lock); - return _intolerable_failure_status; -} - -void IndexChannel::set_error_tablet_in_state(RuntimeState* state) { - std::vector& error_tablet_infos = state->error_tablet_infos(); - - std::lock_guard l(_fail_lock); - for (const auto& it : _failed_channels_msgs) { - TErrorTabletInfo error_info; - error_info.__set_tabletId(it.first); - error_info.__set_msg(it.second); - error_tablet_infos.emplace_back(error_info); - } -} - -void IndexChannel::set_tablets_received_rows( - const std::vector>& tablets_received_rows, int64_t node_id) { - for (const auto& [tablet_id, rows_num] : tablets_received_rows) { - _tablets_received_rows[tablet_id].emplace_back(node_id, rows_num); - } -} - -void IndexChannel::set_tablets_filtered_rows( - const std::vector>& tablets_filtered_rows, int64_t node_id) { - for (const auto& [tablet_id, rows_num] : tablets_filtered_rows) { - _tablets_filtered_rows[tablet_id].emplace_back(node_id, rows_num); - } -} - -Status IndexChannel::check_tablet_received_rows_consistency() { - for (auto& tablet : _tablets_received_rows) { - for (size_t i = 0; i < tablet.second.size(); i++) { - VLOG_NOTICE << "check_tablet_received_rows_consistency, load_id: " << _parent->_load_id - << ", txn_id: " << std::to_string(_parent->_txn_id) - << ", tablet_id: " << tablet.first - << ", node_id: " << tablet.second[i].first - << ", rows_num: " << tablet.second[i].second; - if (i == 0) { - continue; - } - if (tablet.second[i].second != tablet.second[0].second) { - return Status::InternalError( - "rows num written by multi replicas doest't match, load_id={}, txn_id={}, " - "tablt_id={}, node_id={}, rows_num={}, node_id={}, rows_num={}", - print_id(_parent->_load_id), _parent->_txn_id, tablet.first, - tablet.second[i].first, tablet.second[i].second, tablet.second[0].first, - tablet.second[0].second); - } - } - } - return Status::OK(); -} - -Status IndexChannel::check_tablet_filtered_rows_consistency() { - for (auto& tablet : _tablets_filtered_rows) { - for (size_t i = 0; i < tablet.second.size(); i++) { - VLOG_NOTICE << "check_tablet_filtered_rows_consistency, load_id: " << _parent->_load_id - << ", txn_id: " << std::to_string(_parent->_txn_id) - << ", tablet_id: " << tablet.first - << ", node_id: " << tablet.second[i].first - << ", rows_num: " << tablet.second[i].second; - if (i == 0) { - continue; - } - if (tablet.second[i].second != tablet.second[0].second) { - return Status::InternalError( - "rows num filtered by multi replicas doest't match, load_id={}, txn_id={}, " - "tablt_id={}, node_id={}, rows_num={}, node_id={}, rows_num={}", - print_id(_parent->_load_id), _parent->_txn_id, tablet.first, - tablet.second[i].first, tablet.second[i].second, tablet.second[0].first, - tablet.second[0].second); - } - } - } - return Status::OK(); -} - -VNodeChannel::VNodeChannel(VOlapTableSink* parent, IndexChannel* index_channel, int64_t node_id, - bool is_incremental) - : _parent(parent), - _index_channel(index_channel), - _node_id(node_id), - _is_incremental(is_incremental) { - _node_channel_tracker = std::make_shared(fmt::format( - "NodeChannel:indexID={}:threadId={}", std::to_string(_index_channel->_index_id), - thread_context()->get_thread_id())); -} - -VNodeChannel::~VNodeChannel() { - for (auto& closure : _open_closures) { - if (closure != nullptr) { - if (closure->unref()) { - delete closure; - } - closure = nullptr; - } - } - if (_add_block_closure != nullptr) { - delete _add_block_closure; - _add_block_closure = nullptr; - } - static_cast(_cur_add_block_request.release_id()); -} - -void VNodeChannel::clear_all_blocks() { - std::lock_guard lg(_pending_batches_lock); - std::queue empty; - std::swap(_pending_blocks, empty); - _cur_mutable_block.reset(); -} - -// if "_cancelled" is set to true, -// no need to set _cancel_msg because the error will be -// returned directly via "TabletSink::prepare()" method. -Status VNodeChannel::init(RuntimeState* state) { - SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); - _tuple_desc = _parent->_output_tuple_desc; - _state = state; - // get corresponding BE node. - auto node = _parent->_nodes_info->find_node(_node_id); - if (node == nullptr) { - _cancelled = true; - return Status::InternalError("unknown node id, id={}", _node_id); - } - _node_info = *node; - - _load_info = "load_id=" + print_id(_parent->_load_id) + - ", txn_id=" + std::to_string(_parent->_txn_id); - - _row_desc.reset(new RowDescriptor(_tuple_desc, false)); - _batch_size = state->batch_size(); - - _stub = state->exec_env()->brpc_internal_client_cache()->get_client(_node_info.host, - _node_info.brpc_port); - if (_stub == nullptr) { - _cancelled = true; - return Status::InternalError("Get rpc stub failed, host={}, port={}, info={}", - _node_info.host, _node_info.brpc_port, channel_info()); - } - - _rpc_timeout_ms = state->execution_timeout() * 1000; - _timeout_watch.start(); - - // Initialize _cur_add_block_request - if (!_cur_add_block_request.has_id()) { - _cur_add_block_request.set_allocated_id(&_parent->_load_id); - } - _cur_add_block_request.set_index_id(_index_channel->_index_id); - _cur_add_block_request.set_sender_id(_parent->_sender_id); - _cur_add_block_request.set_backend_id(_node_id); - _cur_add_block_request.set_eos(false); - - _name = fmt::format("VNodeChannel[{}-{}]", _index_channel->_index_id, _node_id); - // The node channel will send _batch_size rows of data each rpc. When the - // number of tablets is large, the number of data rows received by each - // tablet is small, TabletsChannel need to traverse each tablet for import. - // so the import performance is poor. Therefore, we set _batch_size to - // a relatively large value to improve the import performance. - _batch_size = std::max(_batch_size, 8192); - - return Status::OK(); -} - -void VNodeChannel::_open_internal(bool is_incremental) { - SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); - PTabletWriterOpenRequest request; - request.set_allocated_id(&_parent->_load_id); - request.set_index_id(_index_channel->_index_id); - request.set_txn_id(_parent->_txn_id); - request.set_allocated_schema(_parent->_schema->to_protobuf()); - std::set deduper; - for (auto& tablet : _all_tablets) { - if (deduper.contains(tablet.tablet_id)) { - continue; - } - auto ptablet = request.add_tablets(); - ptablet->set_partition_id(tablet.partition_id); - ptablet->set_tablet_id(tablet.tablet_id); - deduper.insert(tablet.tablet_id); - } - request.set_num_senders(_parent->_num_senders); - request.set_need_gen_rollup(false); // Useless but it is a required field in pb - request.set_load_mem_limit(_parent->_load_mem_limit); - request.set_load_channel_timeout_s(_parent->_load_channel_timeout_s); - request.set_is_high_priority(_parent->_is_high_priority); - request.set_sender_ip(BackendOptions::get_localhost()); - request.set_is_vectorized(true); - request.set_backend_id(_node_id); - request.set_enable_profile(_state->enable_profile()); - request.set_is_incremental(is_incremental); - - auto* open_closure = new RefCountClosure {}; - open_closure->ref(); - - open_closure->ref(); // This ref is for RPC's reference - open_closure->cntl.set_timeout_ms(config::tablet_writer_open_rpc_timeout_sec * 1000); - if (config::tablet_writer_ignore_eovercrowded) { - open_closure->cntl.ignore_eovercrowded(); - } - // the real transmission here. the corresponding BE's load mgr will open load channel for it. - _stub->tablet_writer_open(&open_closure->cntl, &request, &open_closure->result, open_closure); - _open_closures.push_back(open_closure); - - static_cast(request.release_id()); - static_cast(request.release_schema()); -} - -void VNodeChannel::open() { - _open_internal(false); -} - -void VNodeChannel::incremental_open() { - _open_internal(true); -} - -Status VNodeChannel::open_wait() { - Status status; - for (auto& open_closure : _open_closures) { - // because of incremental open, we will wait multi times. so skip the closures which have been checked and set to nullptr in previous rounds - if (open_closure == nullptr) { - continue; - } - - open_closure->join(); - SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); - if (open_closure->cntl.Failed()) { - if (!ExecEnv::GetInstance()->brpc_internal_client_cache()->available( - _stub, _node_info.host, _node_info.brpc_port)) { - ExecEnv::GetInstance()->brpc_internal_client_cache()->erase( - open_closure->cntl.remote_side()); - } - - _cancelled = true; - auto error_code = open_closure->cntl.ErrorCode(); - auto error_text = open_closure->cntl.ErrorText(); - if (open_closure->unref()) { - delete open_closure; - } - open_closure = nullptr; - return Status::InternalError( - "failed to open tablet writer, error={}, error_text={}, info={}", - berror(error_code), error_text, channel_info()); - } - status = Status::create(open_closure->result.status()); - if (open_closure->unref()) { - delete open_closure; - } - open_closure = nullptr; - - if (!status.ok()) { - _cancelled = true; - return status; - } - } - - // add block closure - _add_block_closure = ReusableClosure::create(); - _add_block_closure->addFailedHandler( - [this](bool is_last_rpc) { _add_block_failed_callback(is_last_rpc); }); - - _add_block_closure->addSuccessHandler( - [this](const PTabletWriterAddBlockResult& result, bool is_last_rpc) { - _add_block_success_callback(result, is_last_rpc); - }); - return status; -} - -Status VNodeChannel::add_block(vectorized::Block* block, const Payload* payload, bool is_append) { - SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); - if (payload->second.empty()) { - return Status::OK(); - } - // If add_block() when _eos_is_produced==true, there must be sth wrong, we can only mark this channel as failed. - auto st = none_of({_cancelled, _eos_is_produced}); - if (!st.ok()) { - if (_cancelled) { - std::lock_guard l(_cancel_msg_lock); - return Status::InternalError("add row failed. {}", _cancel_msg); - } else { - return std::move(st.prepend("already stopped, can't add row. cancelled/eos: ")); - } - } - - // We use OlapTableSink mem_tracker which has the same ancestor of _plan node, - // so in the ideal case, mem limit is a matter for _plan node. - // But there is still some unfinished things, we do mem limit here temporarily. - // _cancelled may be set by rpc callback, and it's possible that _cancelled might be set in any of the steps below. - // It's fine to do a fake add_block() and return OK, because we will check _cancelled in next add_block() or mark_close(). - while (!_cancelled && _pending_batches_num > 0 && - _pending_batches_bytes > _max_pending_batches_bytes) { - SCOPED_RAW_TIMER(&_stat.mem_exceeded_block_ns); - std::this_thread::sleep_for(std::chrono::milliseconds(10)); - } - - if (UNLIKELY(!_cur_mutable_block)) { - _cur_mutable_block = vectorized::MutableBlock::create_unique(block->clone_empty()); - } - - std::unique_ptr temp_payload = nullptr; - if (_index_channel != nullptr && _index_channel->get_where_clause() != nullptr) { - SCOPED_RAW_TIMER(&_stat.where_clause_ns); - temp_payload.reset(new Payload( - std::unique_ptr(new vectorized::IColumn::Selector()), - std::vector())); - int result_index = -1; - size_t column_number = block->columns(); - RETURN_IF_ERROR(_index_channel->get_where_clause()->execute(block, &result_index)); - - auto& row_ids = *payload->first; - auto& tablets_ids = payload->second; - - auto filter_column = block->get_by_position(result_index).column; - - if (auto* nullable_column = - vectorized::check_and_get_column(*filter_column)) { - for (size_t i = 0; i < payload->second.size(); i++) { - if (nullable_column->get_bool_inline(row_ids[i])) { - temp_payload->first->emplace_back(row_ids[i]); - temp_payload->second.emplace_back(tablets_ids[i]); - } - } - payload = temp_payload.get(); - } else if (auto* const_column = vectorized::check_and_get_column( - *filter_column)) { - bool ret = const_column->get_bool(0); - if (!ret) { - return Status::OK(); - } - } else { - auto& filter = assert_cast(*filter_column).get_data(); - for (size_t i = 0; i < payload->second.size(); i++) { - if (filter[row_ids[i]] != 0) { - temp_payload->first->emplace_back(row_ids[i]); - temp_payload->second.emplace_back(tablets_ids[i]); - } - } - payload = temp_payload.get(); - } - - for (size_t i = block->columns() - 1; i >= column_number; i--) { - block->erase(i); - } - } - - SCOPED_RAW_TIMER(&_stat.append_node_channel_ns); - if (is_append) { - // Do not split the data of the block by tablets but append it to a single delta writer. - // This is a faster way to send block than append_to_block_by_selector - // TODO: we could write to local delta writer if single_replica_load is true - VLOG_DEBUG << "send whole block by append block"; - std::vector tablets(block->rows(), payload->second[0]); - vectorized::MutableColumns& columns = _cur_mutable_block->mutable_columns(); - columns.clear(); - columns.reserve(block->columns()); - // Hold the reference of block columns to avoid copying - for (auto column : block->get_columns()) { - columns.push_back(std::move(*column).mutate()); - } - *_cur_add_block_request.mutable_tablet_ids() = {tablets.begin(), tablets.end()}; - _cur_add_block_request.set_is_single_tablet_block(true); - } else { - block->append_to_block_by_selector(_cur_mutable_block.get(), *(payload->first)); - for (auto tablet_id : payload->second) { - _cur_add_block_request.add_tablet_ids(tablet_id); - } - } - - if (is_append || _cur_mutable_block->rows() >= _batch_size || - _cur_mutable_block->bytes() > config::doris_scanner_row_bytes) { - { - SCOPED_ATOMIC_TIMER(&_queue_push_lock_ns); - std::lock_guard l(_pending_batches_lock); - // To simplify the add_row logic, postpone adding block into req until the time of sending req - _pending_batches_bytes += _cur_mutable_block->allocated_bytes(); - _cur_add_block_request.set_eos( - false); // for multi-add, only when marking close we set it eos. - _pending_blocks.emplace(std::move(_cur_mutable_block), _cur_add_block_request); - _pending_batches_num++; - VLOG_DEBUG << "VOlapTableSink:" << _parent << " VNodeChannel:" << this - << " pending_batches_bytes:" << _pending_batches_bytes - << " jobid:" << std::to_string(_state->load_job_id()) - << " loadinfo:" << _load_info; - } - _cur_mutable_block = vectorized::MutableBlock::create_unique(block->clone_empty()); - _cur_add_block_request.clear_tablet_ids(); - } - - return Status::OK(); -} - -int VNodeChannel::try_send_and_fetch_status(RuntimeState* state, - std::unique_ptr& thread_pool_token) { - auto st = none_of({_cancelled, _send_finished}); - if (!st.ok()) { - return 0; - } - - if (!_add_block_closure->try_set_in_flight()) { - // There is packet in flight, skip. - return _send_finished ? 0 : 1; - } - - // We are sure that try_send_batch is not running - if (_pending_batches_num > 0) { - auto s = thread_pool_token->submit_func( - std::bind(&VNodeChannel::try_send_pending_block, this, state)); - if (!s.ok()) { - _cancel_with_msg("submit send_batch task to send_batch_thread_pool failed"); - // clear in flight - _add_block_closure->clear_in_flight(); - } - // in_flight is cleared in closure::Run - } else { - // clear in flight - _add_block_closure->clear_in_flight(); - } - return _send_finished ? 0 : 1; -} - -void VNodeChannel::_cancel_with_msg(const std::string& msg) { - LOG(WARNING) << "cancel node channel " << channel_info() << ", error message: " << msg; - { - std::lock_guard l(_cancel_msg_lock); - if (_cancel_msg == "") { - _cancel_msg = msg; - } - } - _cancelled = true; -} - -Status VNodeChannel::none_of(std::initializer_list vars) { - bool none = std::none_of(vars.begin(), vars.end(), [](bool var) { return var; }); - Status st = Status::OK(); - if (!none) { - std::string vars_str; - std::for_each(vars.begin(), vars.end(), - [&vars_str](bool var) -> void { vars_str += (var ? "1/" : "0/"); }); - if (!vars_str.empty()) { - vars_str.pop_back(); // 0/1/0/ -> 0/1/0 - } - st = Status::Uninitialized(vars_str); - } - - return st; -} - -void VNodeChannel::try_send_pending_block(RuntimeState* state) { - SCOPED_ATTACH_TASK(state); - SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker); - SCOPED_ATOMIC_TIMER(&_actual_consume_ns); - AddBlockReq send_block; - { - std::lock_guard l(_pending_batches_lock); - DCHECK(!_pending_blocks.empty()); - send_block = std::move(_pending_blocks.front()); - _pending_blocks.pop(); - _pending_batches_num--; - _pending_batches_bytes -= send_block.first->allocated_bytes(); - } - - auto mutable_block = std::move(send_block.first); - auto request = std::move(send_block.second); // doesn't need to be saved in heap - - // tablet_ids has already set when add row - request.set_packet_seq(_next_packet_seq); - auto block = mutable_block->to_block(); - CHECK(block.rows() == request.tablet_ids_size()) - << "block rows: " << block.rows() << ", tablet_ids_size: " << request.tablet_ids_size(); - if (block.rows() > 0) { - SCOPED_ATOMIC_TIMER(&_serialize_batch_ns); - size_t uncompressed_bytes = 0, compressed_bytes = 0; - Status st = block.serialize(state->be_exec_version(), request.mutable_block(), - &uncompressed_bytes, &compressed_bytes, - state->fragement_transmission_compression_type(), - _parent->_transfer_large_data_by_brpc); - if (!st.ok()) { - cancel(fmt::format("{}, err: {}", channel_info(), st.to_string())); - _add_block_closure->clear_in_flight(); - return; - } - if (compressed_bytes >= double(config::brpc_max_body_size) * 0.95f) { - LOG(WARNING) << "send block too large, this rpc may failed. send size: " - << compressed_bytes << ", threshold: " << config::brpc_max_body_size - << ", " << channel_info(); - } - } - - int remain_ms = _rpc_timeout_ms - _timeout_watch.elapsed_time() / NANOS_PER_MILLIS; - if (UNLIKELY(remain_ms < config::min_load_rpc_timeout_ms)) { - if (remain_ms <= 0 && !request.eos()) { - cancel(fmt::format("{}, err: timeout", channel_info())); - _add_block_closure->clear_in_flight(); - return; - } else { - remain_ms = config::min_load_rpc_timeout_ms; - } - } - - _add_block_closure->reset(); - _add_block_closure->cntl.set_timeout_ms(remain_ms); - if (config::tablet_writer_ignore_eovercrowded) { - _add_block_closure->cntl.ignore_eovercrowded(); - } - - if (request.eos()) { - for (auto pid : _parent->_tablet_finder->partition_ids()) { - request.add_partition_ids(pid); - } - - request.set_write_single_replica(false); - if (_parent->_write_single_replica) { - request.set_write_single_replica(true); - for (std::unordered_map>::iterator iter = - _slave_tablet_nodes.begin(); - iter != _slave_tablet_nodes.end(); iter++) { - PSlaveTabletNodes slave_tablet_nodes; - for (auto node_id : iter->second) { - auto node = _parent->_nodes_info->find_node(node_id); - if (node == nullptr) { - return; - } - PNodeInfo* pnode = slave_tablet_nodes.add_slave_nodes(); - pnode->set_id(node->id); - pnode->set_option(node->option); - pnode->set_host(node->host); - pnode->set_async_internal_port(node->brpc_port); - } - request.mutable_slave_tablet_nodes()->insert({iter->first, slave_tablet_nodes}); - } - } - - // eos request must be the last request - _add_block_closure->end_mark(); - _send_finished = true; - CHECK(_pending_batches_num == 0) << _pending_batches_num; - } - - if (_parent->_transfer_large_data_by_brpc && request.has_block() && - request.block().has_column_values() && request.ByteSizeLong() > MIN_HTTP_BRPC_SIZE) { - Status st = request_embed_attachment_contain_block< - PTabletWriterAddBlockRequest, ReusableClosure>( - &request, _add_block_closure); - if (!st.ok()) { - cancel(fmt::format("{}, err: {}", channel_info(), st.to_string())); - _add_block_closure->clear_in_flight(); - return; - } - - //format an ipv6 address - std::string brpc_url = get_brpc_http_url(_node_info.host, _node_info.brpc_port); - std::shared_ptr _brpc_http_stub = - _state->exec_env()->brpc_internal_client_cache()->get_new_client_no_cache(brpc_url, - "http"); - _add_block_closure->cntl.http_request().uri() = - brpc_url + "/PInternalServiceImpl/tablet_writer_add_block_by_http"; - _add_block_closure->cntl.http_request().set_method(brpc::HTTP_METHOD_POST); - _add_block_closure->cntl.http_request().set_content_type("application/json"); - - { - SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker()); - _brpc_http_stub->tablet_writer_add_block_by_http(&_add_block_closure->cntl, nullptr, - &_add_block_closure->result, - _add_block_closure); - } - } else { - _add_block_closure->cntl.http_request().Clear(); - { - SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker()); - _stub->tablet_writer_add_block(&_add_block_closure->cntl, &request, - &_add_block_closure->result, _add_block_closure); - } - } - - _next_packet_seq++; -} - -void VNodeChannel::_add_block_success_callback(const PTabletWriterAddBlockResult& result, - bool is_last_rpc) { - std::lock_guard l(this->_closed_lock); - if (this->_is_closed) { - // if the node channel is closed, no need to call the following logic, - // and notice that _index_channel may already be destroyed. - return; - } - SCOPED_ATTACH_TASK(_state); - Status status(Status::create(result.status())); - if (status.ok()) { - // if has error tablet, handle them first - for (auto& error : result.tablet_errors()) { - _index_channel->mark_as_failed(this, "tablet error: " + error.msg(), error.tablet_id()); - } - - Status st = _index_channel->check_intolerable_failure(); - if (!st.ok()) { - _cancel_with_msg(st.to_string()); - } else if (is_last_rpc) { - for (auto& tablet : result.tablet_vec()) { - TTabletCommitInfo commit_info; - commit_info.tabletId = tablet.tablet_id(); - commit_info.backendId = _node_id; - _tablet_commit_infos.emplace_back(std::move(commit_info)); - if (tablet.has_received_rows()) { - _tablets_received_rows.emplace_back(tablet.tablet_id(), tablet.received_rows()); - } - if (tablet.has_num_rows_filtered()) { - _tablets_filtered_rows.emplace_back(tablet.tablet_id(), - tablet.num_rows_filtered()); - } - VLOG_CRITICAL << "master replica commit info: tabletId=" << tablet.tablet_id() - << ", backendId=" << _node_id - << ", master node id: " << this->node_id() - << ", host: " << this->host() << ", txn_id=" << _parent->_txn_id; - } - if (_parent->_write_single_replica) { - for (auto& tablet_slave_node_ids : result.success_slave_tablet_node_ids()) { - for (auto slave_node_id : tablet_slave_node_ids.second.slave_node_ids()) { - TTabletCommitInfo commit_info; - commit_info.tabletId = tablet_slave_node_ids.first; - commit_info.backendId = slave_node_id; - _tablet_commit_infos.emplace_back(std::move(commit_info)); - VLOG_CRITICAL - << "slave replica commit info: tabletId=" - << tablet_slave_node_ids.first << ", backendId=" << slave_node_id - << ", master node id: " << this->node_id() - << ", host: " << this->host() << ", txn_id=" << _parent->_txn_id; - } - } - } - _add_batches_finished = true; - } - } else { - _cancel_with_msg(fmt::format("{}, add batch req success but status isn't ok, err: {}", - channel_info(), status.to_string())); - } - - if (result.has_execution_time_us()) { - _add_batch_counter.add_batch_execution_time_us += result.execution_time_us(); - _add_batch_counter.add_batch_wait_execution_time_us += result.wait_execution_time_us(); - _add_batch_counter.add_batch_num++; - } - if (result.has_load_channel_profile()) { - TRuntimeProfileTree tprofile; - const uint8_t* buf = (const uint8_t*)result.load_channel_profile().data(); - uint32_t len = result.load_channel_profile().size(); - auto st = deserialize_thrift_msg(buf, &len, false, &tprofile); - if (st.ok()) { - _state->load_channel_profile()->update(tprofile); - } else { - LOG(WARNING) << "load channel TRuntimeProfileTree deserialize failed, errmsg=" << st; - } - } -} - -void VNodeChannel::_add_block_failed_callback(bool is_last_rpc) { - std::lock_guard l(this->_closed_lock); - if (this->_is_closed) { - // if the node channel is closed, no need to call `mark_as_failed`, - // and notice that _index_channel may already be destroyed. - return; - } - SCOPED_ATTACH_TASK(_state); - // If rpc failed, mark all tablets on this node channel as failed - _index_channel->mark_as_failed( - this, - fmt::format("rpc failed, error coed:{}, error text:{}", - _add_block_closure->cntl.ErrorCode(), _add_block_closure->cntl.ErrorText()), - -1); - Status st = _index_channel->check_intolerable_failure(); - if (!st.ok()) { - _cancel_with_msg(fmt::format("{}, err: {}", channel_info(), st.to_string())); - } else if (is_last_rpc) { - // if this is last rpc, will must set _add_batches_finished. otherwise, node channel's close_wait - // will be blocked. - _add_batches_finished = true; - } -} - -void VNodeChannel::cancel(const std::string& cancel_msg) { - if (_is_closed) { - // skip the channels that have been canceled or close_wait. - return; - } - SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); - // set _is_closed to true finally - Defer set_closed {[&]() { - std::lock_guard l(_closed_lock); - _is_closed = true; - }}; - // we don't need to wait last rpc finished, cause closure's release/reset will join. - // But do we need brpc::StartCancel(call_id)? - _cancel_with_msg(cancel_msg); - - PTabletWriterCancelRequest request; - request.set_allocated_id(&_parent->_load_id); - request.set_index_id(_index_channel->_index_id); - request.set_sender_id(_parent->_sender_id); - - auto closure = new RefCountClosure(); - - closure->ref(); - int remain_ms = _rpc_timeout_ms - _timeout_watch.elapsed_time() / NANOS_PER_MILLIS; - if (UNLIKELY(remain_ms < config::min_load_rpc_timeout_ms)) { - remain_ms = config::min_load_rpc_timeout_ms; - } - closure->cntl.set_timeout_ms(remain_ms); - if (config::tablet_writer_ignore_eovercrowded) { - closure->cntl.ignore_eovercrowded(); - } - _stub->tablet_writer_cancel(&closure->cntl, &request, &closure->result, closure); - static_cast(request.release_id()); -} - -bool VNodeChannel::is_send_data_rpc_done() const { - return _add_batches_finished || _cancelled; -} - -Status VNodeChannel::close_wait(RuntimeState* state) { - SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); - // set _is_closed to true finally - Defer set_closed {[&]() { - std::lock_guard l(_closed_lock); - _is_closed = true; - }}; - - auto st = none_of({_cancelled, !_eos_is_produced}); - if (!st.ok()) { - if (_cancelled) { - std::lock_guard l(_cancel_msg_lock); - return Status::InternalError("wait close failed. {}", _cancel_msg); - } else { - return std::move( - st.prepend("already stopped, skip waiting for close. cancelled/!eos: ")); - } - } - - // waiting for finished, it may take a long time, so we couldn't set a timeout - // In pipeline, is_close_done() is false at this time, will not bock. - while (!_add_batches_finished && !_cancelled && !state->is_cancelled()) { - bthread_usleep(1000); - } - _close_time_ms = UnixMillis() - _close_time_ms; - - if (_add_batches_finished) { - _close_check(); - state->tablet_commit_infos().insert(state->tablet_commit_infos().end(), - std::make_move_iterator(_tablet_commit_infos.begin()), - std::make_move_iterator(_tablet_commit_infos.end())); - - _index_channel->set_error_tablet_in_state(state); - _index_channel->set_tablets_received_rows(_tablets_received_rows, _node_id); - _index_channel->set_tablets_filtered_rows(_tablets_filtered_rows, _node_id); - return Status::OK(); - } - - return Status::InternalError(get_cancel_msg()); -} - -void VNodeChannel::_close_check() { - std::lock_guard lg(_pending_batches_lock); - CHECK(_pending_blocks.empty()) << name(); - CHECK(_cur_mutable_block == nullptr) << name(); -} - -void VNodeChannel::mark_close() { - auto st = none_of({_cancelled, _eos_is_produced}); - if (!st.ok()) { - return; - } - - _cur_add_block_request.set_eos(true); - { - std::lock_guard l(_pending_batches_lock); - if (!_cur_mutable_block) { - // add a dummy block - _cur_mutable_block = vectorized::MutableBlock::create_unique(); - } - // when prepare to close, add block to queue so that try_send_pending_block thread will send it. - _pending_blocks.emplace(std::move(_cur_mutable_block), _cur_add_block_request); - _pending_batches_num++; - DCHECK(_pending_blocks.back().second.eos()); - _close_time_ms = UnixMillis(); - LOG(INFO) << channel_info() - << " mark closed, left pending batch size: " << _pending_blocks.size(); - } - - _eos_is_produced = true; -} +namespace vectorized { VOlapTableSink::VOlapTableSink(ObjectPool* pool, const RowDescriptor& row_desc, - const std::vector& texprs, Status* status) - : DataSink(row_desc), _pool(pool) { - // From the thrift expressions create the real exprs. - *status = vectorized::VExpr::create_expr_trees(texprs, _output_vexpr_ctxs); - _name = "VOlapTableSink"; - _transfer_large_data_by_brpc = config::transfer_large_data_by_brpc; -} - -VOlapTableSink::~VOlapTableSink() { - // We clear NodeChannels' batches here, cuz NodeChannels' batches destruction will use - // OlapTableSink::_mem_tracker and its parents. - // But their destructions are after OlapTableSink's. - for (const auto& index_channel : _channels) { - index_channel->for_each_node_channel( - [](const std::shared_ptr& ch) { ch->clear_all_blocks(); }); - } -} + const std::vector& texprs, bool group_commit) + : AsyncWriterSink(row_desc, texprs), + _pool(pool), + _group_commit(group_commit) {} Status VOlapTableSink::init(const TDataSink& t_sink) { - DCHECK(t_sink.__isset.olap_table_sink); - auto& table_sink = t_sink.olap_table_sink; - _load_id.set_hi(table_sink.load_id.hi); - _load_id.set_lo(table_sink.load_id.lo); - _txn_id = table_sink.txn_id; - _num_replicas = table_sink.num_replicas; - _tuple_desc_id = table_sink.tuple_id; - _schema.reset(new OlapTableSchemaParam()); - RETURN_IF_ERROR(_schema->init(table_sink.schema)); - _location = _pool->add(new OlapTableLocationParam(table_sink.location)); - _nodes_info = _pool->add(new DorisNodesInfo(table_sink.nodes_info)); - if (table_sink.__isset.write_single_replica && table_sink.write_single_replica) { - _write_single_replica = true; - _slave_location = _pool->add(new OlapTableLocationParam(table_sink.slave_location)); - if (!config::enable_single_replica_load) { - return Status::InternalError("single replica load is disabled on BE."); - } - } - - if (table_sink.__isset.load_channel_timeout_s) { - _load_channel_timeout_s = table_sink.load_channel_timeout_s; - } else { - _load_channel_timeout_s = config::streaming_load_rpc_max_alive_time_sec; - } - if (table_sink.__isset.send_batch_parallelism && table_sink.send_batch_parallelism > 1) { - _send_batch_parallelism = table_sink.send_batch_parallelism; - } - // if distributed column list is empty, we can ensure that tablet is with random distribution info - // and if load_to_single_tablet is set and set to true, we should find only one tablet in one partition - // for the whole olap table sink - auto find_tablet_mode = OlapTabletFinder::FindTabletMode::FIND_TABLET_EVERY_ROW; - if (table_sink.partition.distributed_columns.empty()) { - if (table_sink.__isset.load_to_single_tablet && table_sink.load_to_single_tablet) { - find_tablet_mode = OlapTabletFinder::FindTabletMode::FIND_TABLET_EVERY_SINK; - } else { - find_tablet_mode = OlapTabletFinder::FindTabletMode::FIND_TABLET_EVERY_BATCH; - } - } - _vpartition = _pool->add(new doris::VOlapTablePartitionParam(_schema, table_sink.partition)); - _tablet_finder = std::make_unique(_vpartition, find_tablet_mode); - return _vpartition->init(); -} - -Status VOlapTableSink::prepare(RuntimeState* state) { - RETURN_IF_ERROR(DataSink::prepare(state)); - - _state = state; - - _sender_id = state->per_fragment_instance_idx(); - _num_senders = state->num_per_fragment_instances(); - _is_high_priority = - (state->execution_timeout() <= config::load_task_high_priority_threshold_second); - - // profile must add to state's object pool - _profile = state->obj_pool()->add(new RuntimeProfile("OlapTableSink")); - _mem_tracker = - std::make_shared("OlapTableSink:" + std::to_string(state->load_job_id())); - SCOPED_TIMER(_profile->total_time_counter()); - SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); - - // get table's tuple descriptor - _output_tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_desc_id); - if (_output_tuple_desc == nullptr) { - LOG(WARNING) << "unknown destination tuple descriptor, id=" << _tuple_desc_id; - return Status::InternalError("unknown destination tuple descriptor"); - } - - _block_convertor = std::make_unique(_output_tuple_desc); - _block_convertor->init_autoinc_info(_schema->db_id(), _schema->table_id(), - _state->batch_size()); - _output_row_desc = _pool->add(new RowDescriptor(_output_tuple_desc, false)); - - // add all counter - _input_rows_counter = ADD_COUNTER(_profile, "RowsRead", TUnit::UNIT); - _output_rows_counter = ADD_COUNTER(_profile, "RowsReturned", TUnit::UNIT); - _filtered_rows_counter = ADD_COUNTER(_profile, "RowsFiltered", TUnit::UNIT); - _send_data_timer = ADD_TIMER(_profile, "SendDataTime"); - _wait_mem_limit_timer = ADD_CHILD_TIMER(_profile, "WaitMemLimitTime", "SendDataTime"); - _row_distribution_timer = ADD_CHILD_TIMER(_profile, "RowDistributionTime", "SendDataTime"); - _filter_timer = ADD_CHILD_TIMER(_profile, "FilterTime", "SendDataTime"); - _where_clause_timer = ADD_CHILD_TIMER(_profile, "WhereClauseTime", "SendDataTime"); - _append_node_channel_timer = ADD_CHILD_TIMER(_profile, "AppendNodeChannelTime", "SendDataTime"); - _add_partition_request_timer = - ADD_CHILD_TIMER(_profile, "AddPartitionRequestTime", "SendDataTime"); - _validate_data_timer = ADD_TIMER(_profile, "ValidateDataTime"); - _open_timer = ADD_TIMER(_profile, "OpenTime"); - _close_timer = ADD_TIMER(_profile, "CloseWaitTime"); - _non_blocking_send_timer = ADD_TIMER(_profile, "NonBlockingSendTime"); - _non_blocking_send_work_timer = - ADD_CHILD_TIMER(_profile, "NonBlockingSendWorkTime", "NonBlockingSendTime"); - _serialize_batch_timer = - ADD_CHILD_TIMER(_profile, "SerializeBatchTime", "NonBlockingSendWorkTime"); - _total_add_batch_exec_timer = ADD_TIMER(_profile, "TotalAddBatchExecTime"); - _max_add_batch_exec_timer = ADD_TIMER(_profile, "MaxAddBatchExecTime"); - _total_wait_exec_timer = ADD_TIMER(_profile, "TotalWaitExecTime"); - _max_wait_exec_timer = ADD_TIMER(_profile, "MaxWaitExecTime"); - _add_batch_number = ADD_COUNTER(_profile, "NumberBatchAdded", TUnit::UNIT); - _num_node_channels = ADD_COUNTER(_profile, "NumberNodeChannels", TUnit::UNIT); - _load_mem_limit = state->get_load_mem_limit(); - -#ifdef DEBUG - // check: tablet ids should be unique - { - std::unordered_set tablet_ids; - const auto& partitions = _vpartition->get_partitions(); - for (int i = 0; i < _schema->indexes().size(); ++i) { - for (const auto& partition : partitions) { - for (const auto& tablet : partition->indexes[i].tablets) { - CHECK(tablet_ids.count(tablet) == 0) << "found duplicate tablet id: " << tablet; - tablet_ids.insert(tablet); - } - } - } - } -#endif - - // open all channels - const auto& partitions = _vpartition->get_partitions(); - for (int i = 0; i < _schema->indexes().size(); ++i) { - // collect all tablets belong to this rollup - std::vector tablets; - auto index = _schema->indexes()[i]; - for (const auto& part : partitions) { - for (const auto& tablet : part->indexes[i].tablets) { - TTabletWithPartition tablet_with_partition; - tablet_with_partition.partition_id = part->id; - tablet_with_partition.tablet_id = tablet; - tablets.emplace_back(std::move(tablet_with_partition)); - } - } - if (tablets.empty() && !_vpartition->is_auto_partition()) { - LOG(WARNING) << "load job:" << state->load_job_id() << " index: " << index->index_id - << " would open 0 tablet"; - } - _channels.emplace_back(new IndexChannel(this, index->index_id, index->where_clause)); - _index_id_to_channel[index->index_id] = _channels.back(); - RETURN_IF_ERROR(_channels.back()->init(state, tablets)); - } - // Prepare the exprs to run. - RETURN_IF_ERROR(vectorized::VExpr::prepare(_output_vexpr_ctxs, state, _row_desc)); - // prepare for auto partition functions - if (_vpartition->is_auto_partition()) { - auto [part_ctx, part_func] = _get_partition_function(); - RETURN_IF_ERROR(part_func->prepare(_state, *_output_row_desc, part_ctx.get())); - } - - _prepare = true; + RETURN_IF_ERROR(AsyncWriterSink::init(t_sink)); + _writer->init_properties(_pool, _group_commit); return Status::OK(); } -static void* periodic_send_batch(void* sink) { - VOlapTableSink* vsink = (VOlapTableSink*)sink; - vsink->_send_batch_process(); - return nullptr; -} - -Status VOlapTableSink::open(RuntimeState* state) { - // Prepare the exprs to run. - RETURN_IF_ERROR(vectorized::VExpr::open(_output_vexpr_ctxs, state)); - SCOPED_TIMER(_profile->total_time_counter()); - SCOPED_TIMER(_open_timer); - SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); - - fmt::memory_buffer buf; - for (const auto& index_channel : _channels) { - fmt::format_to(buf, "index id:{}", index_channel->_index_id); - index_channel->for_each_node_channel( - [](const std::shared_ptr& ch) { ch->open(); }); - } - VLOG_DEBUG << "list of open index id = " << fmt::to_string(buf); - - for (const auto& index_channel : _channels) { - index_channel->for_each_node_channel([&index_channel]( - const std::shared_ptr& ch) { - auto st = ch->open_wait(); - if (!st.ok()) { - // The open() phase is mainly to generate DeltaWriter instances on the nodes corresponding to each node channel. - // This phase will not fail due to a single tablet. - // Therefore, if the open() phase fails, all tablets corresponding to the node need to be marked as failed. - index_channel->mark_as_failed( - ch.get(), - fmt::format("{}, open failed, err: {}", ch->channel_info(), st.to_string()), - -1); - } - }); - - RETURN_IF_ERROR(index_channel->check_intolerable_failure()); - } - int32_t send_batch_parallelism = - MIN(_send_batch_parallelism, config::max_send_batch_parallelism_per_job); - _send_batch_thread_pool_token = state->exec_env()->send_batch_thread_pool()->new_token( - ThreadPool::ExecutionMode::CONCURRENT, send_batch_parallelism); - // start to send batch continually - if (bthread_start_background(&_sender_thread, nullptr, periodic_send_batch, (void*)this) != 0) { - return Status::Error("bthread_start_backgroud failed"); - } - - return Status::OK(); -} - -void VOlapTableSink::_send_batch_process() { - SCOPED_TIMER(_non_blocking_send_timer); - SCOPED_ATTACH_TASK(_state); - SCOPED_CONSUME_MEM_TRACKER(_mem_tracker); - - bool had_effect = false; - while (true) { - // incremental open will temporarily make channels into abnormal state. stop checking when this. - std::unique_lock l(_stop_check_channel); - - int running_channels_num = 0; - for (const auto& index_channel : _channels) { - index_channel->for_each_node_channel([&running_channels_num, - this](const std::shared_ptr& ch) { - running_channels_num += - ch->try_send_and_fetch_status(_state, this->_send_batch_thread_pool_token); - }); - } - - // if there is no channel, maybe auto partition table. so check does there have had running channels ever. - if (running_channels_num == 0 && had_effect) { - LOG(INFO) << "all node channels are stopped(maybe finished/offending/cancelled), " - "sender thread exit. " - << print_id(_load_id); - return; - } else if (running_channels_num != 0) { - had_effect = true; - } - bthread_usleep(config::olap_table_sink_send_interval_ms * 1000); - } -} - -size_t VOlapTableSink::get_pending_bytes() const { - size_t mem_consumption = 0; - for (const auto& index_channel : _channels) { - mem_consumption += index_channel->get_pending_bytes(); - } - return mem_consumption; -} - -Status VOlapTableSink::_automatic_create_partition() { - SCOPED_TIMER(_add_partition_request_timer); - TCreatePartitionRequest request; - TCreatePartitionResult result; - request.__set_txn_id(_txn_id); - request.__set_db_id(_vpartition->db_id()); - request.__set_table_id(_vpartition->table_id()); - request.__set_partitionValues(_partitions_need_create); - - VLOG(1) << "automatic partition rpc begin request " << request; - TNetworkAddress master_addr = ExecEnv::GetInstance()->master_info()->network_address; - int time_out = _state->execution_timeout() * 1000; - RETURN_IF_ERROR(ThriftRpcHelper::rpc( - master_addr.hostname, master_addr.port, - [&request, &result](FrontendServiceConnection& client) { - client->createPartition(result, request); - }, - time_out)); - - Status status(Status::create(result.status)); - VLOG(1) << "automatic partition rpc end response " << result; - if (result.status.status_code == TStatusCode::OK) { - // add new created partitions - RETURN_IF_ERROR(_vpartition->add_partitions(result.partitions)); - - // add new tablet locations. it will use by address. so add to pool - auto* new_locations = _pool->add(new std::vector(result.tablets)); - _location->add_locations(*new_locations); - - // update new node info - _nodes_info->add_nodes(result.nodes); - - // incremental open node channel - RETURN_IF_ERROR(_incremental_open_node_channel(result.partitions)); - } - - return status; -} - -Status VOlapTableSink::_incremental_open_node_channel( - const std::vector& partitions) { - // do what we did in prepare() for partitions. indexes which don't change when we create new partition is orthogonal to partitions. - std::unique_lock _l(_stop_check_channel); - for (int i = 0; i < _schema->indexes().size(); ++i) { - const OlapTableIndexSchema* index = _schema->indexes()[i]; - std::vector tablets; - for (auto& t_part : partitions) { - VOlapTablePartition* part = nullptr; - RETURN_IF_ERROR(_vpartition->generate_partition_from(t_part, part)); - for (const auto& tablet : part->indexes[i].tablets) { - TTabletWithPartition tablet_with_partition; - tablet_with_partition.partition_id = part->id; - tablet_with_partition.tablet_id = tablet; - tablets.emplace_back(std::move(tablet_with_partition)); - } - DCHECK(!tablets.empty()) << "incremental open got nothing!"; - } - // update and reinit for existing channels. - std::shared_ptr channel = _index_id_to_channel[index->index_id]; - DCHECK(channel != nullptr); - RETURN_IF_ERROR(channel->init(_state, tablets)); // add tablets into it - } - - fmt::memory_buffer buf; - for (auto& channel : _channels) { - // incremental open new partition's tablet on storage side - channel->for_each_node_channel( - [](const std::shared_ptr& ch) { ch->incremental_open(); }); - fmt::format_to(buf, "index id:{}", channel->_index_id); - VLOG_DEBUG << "list of open index id = " << fmt::to_string(buf); - - channel->for_each_node_channel([&channel](const std::shared_ptr& ch) { - auto st = ch->open_wait(); - if (!st.ok()) { - // The open() phase is mainly to generate DeltaWriter instances on the nodes corresponding to each node channel. - // This phase will not fail due to a single tablet. - // Therefore, if the open() phase fails, all tablets corresponding to the node need to be marked as failed. - channel->mark_as_failed( - ch.get(), - fmt::format("{}, open failed, err: {}", ch->channel_info(), st.to_string()), - -1); - } - }); - - RETURN_IF_ERROR(channel->check_intolerable_failure()); - } - - return Status::OK(); -} - -void VOlapTableSink::_generate_row_distribution_payload( - ChannelDistributionPayload& channel_to_payload, const VOlapTablePartition* partition, - uint32_t tablet_index, int row_idx, size_t row_cnt) { - // Generate channel payload for sinking data to differenct node channel - for (int j = 0; j < partition->indexes.size(); ++j) { - auto tid = partition->indexes[j].tablets[tablet_index]; - auto it = _channels[j]->_channels_by_tablet.find(tid); - DCHECK(it != _channels[j]->_channels_by_tablet.end()) - << "unknown tablet, tablet_id=" << tablet_index; - for (const auto& channel : it->second) { - if (channel_to_payload[j].count(channel.get()) < 1) { - channel_to_payload[j].insert( - {channel.get(), Payload {std::unique_ptr( - new vectorized::IColumn::Selector()), - std::vector()}}); - } - channel_to_payload[j][channel.get()].first->push_back(row_idx); - channel_to_payload[j][channel.get()].second.push_back(tid); - } - _number_output_rows += row_cnt; - } -} - -Status VOlapTableSink::_single_partition_generate(RuntimeState* state, vectorized::Block* block, - ChannelDistributionPayload& channel_to_payload, - size_t num_rows, bool has_filtered_rows) { - const VOlapTablePartition* partition = nullptr; - uint32_t tablet_index = 0; - bool stop_processing = false; - for (int32_t i = 0; i < num_rows; ++i) { - if (UNLIKELY(has_filtered_rows) && _block_convertor->filter_bitmap().Get(i)) { - continue; - } - bool is_continue = false; - RETURN_IF_ERROR(_tablet_finder->find_tablet(state, block, i, &partition, tablet_index, - stop_processing, is_continue)); - if (is_continue) { - continue; - } - break; - } - if (partition == nullptr) { - return Status::OK(); - } - for (int j = 0; j < partition->indexes.size(); ++j) { - auto tid = partition->indexes[j].tablets[tablet_index]; - auto it = _channels[j]->_channels_by_tablet.find(tid); - DCHECK(it != _channels[j]->_channels_by_tablet.end()) - << "unknown tablet, tablet_id=" << tablet_index; - int64_t row_cnt = 0; - for (const auto& channel : it->second) { - if (channel_to_payload[j].count(channel.get()) < 1) { - channel_to_payload[j].insert( - {channel.get(), Payload {std::unique_ptr( - new vectorized::IColumn::Selector()), - std::vector()}}); - } - auto& selector = channel_to_payload[j][channel.get()].first; - auto& tablet_ids = channel_to_payload[j][channel.get()].second; - for (int32_t i = 0; i < num_rows; ++i) { - if (UNLIKELY(has_filtered_rows) && _block_convertor->filter_bitmap().Get(i)) { - continue; - } - selector->push_back(i); - } - tablet_ids.resize(selector->size(), tid); - row_cnt = selector->size(); - } - _number_output_rows += row_cnt; - } - return Status::OK(); -} - -std::pair -VOlapTableSink::_get_partition_function() { - return {_vpartition->get_part_func_ctx(), _vpartition->get_partition_function()}; -} - -void VOlapTableSink::_save_missing_values(vectorized::ColumnPtr col, - vectorized::DataTypePtr value_type, - std::vector filter) { - _partitions_need_create.clear(); - std::set deduper; - // de-duplication - for (auto row : filter) { - deduper.emplace(value_type->to_string(*col, row)); - } - for (auto& value : deduper) { - TStringLiteral node; - node.value = value; - _partitions_need_create.emplace_back(std::vector {node}); // only 1 partition column now - } -} - -Status VOlapTableSink::send(RuntimeState* state, vectorized::Block* input_block, bool eos) { - SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); - Status status = Status::OK(); - - if (state->query_options().dry_run_query) { - return status; - } - - auto rows = input_block->rows(); - auto bytes = input_block->bytes(); - if (UNLIKELY(rows == 0)) { - return status; - } - SCOPED_TIMER(_profile->total_time_counter()); - _number_input_rows += rows; - // update incrementally so that FE can get the progress. - // the real 'num_rows_load_total' will be set when sink being closed. - state->update_num_rows_load_total(rows); - state->update_num_bytes_load_total(bytes); - DorisMetrics::instance()->load_rows->increment(rows); - DorisMetrics::instance()->load_bytes->increment(bytes); - - std::shared_ptr block; - bool has_filtered_rows = false; - int64_t filtered_rows = - _block_convertor->num_filtered_rows() + _tablet_finder->num_filtered_rows(); - RETURN_IF_ERROR(_block_convertor->validate_and_convert_block( - state, input_block, block, _output_vexpr_ctxs, rows, eos, has_filtered_rows)); - - SCOPED_RAW_TIMER(&_send_data_ns); - // This is just for passing compilation. - bool stop_processing = false; - std::vector> channel_to_payload; - channel_to_payload.resize(_channels.size()); - _tablet_finder->clear_for_new_batch(); - _row_distribution_watch.start(); - auto num_rows = block->rows(); - size_t partition_num = _vpartition->get_partitions().size(); - if (!_vpartition->is_auto_partition() && partition_num == 1 && - _tablet_finder->is_find_tablet_every_sink()) { - RETURN_IF_ERROR(_single_partition_generate(state, block.get(), channel_to_payload, num_rows, - has_filtered_rows)); - } else { - // if there's projection of partition calc, we need to calc it first. - auto [part_ctx, part_func] = _get_partition_function(); - int result_idx; - if (_vpartition->is_projection_partition()) { - // calc the start value of missing partition ranges. - part_func->execute(part_ctx.get(), block.get(), &result_idx); - VLOG_DEBUG << "Partition-calculated block:" << block->dump_data(); - // change the column to compare to transformed. - _vpartition->set_transformed_slots({(uint16_t)result_idx}); - } - - if (_vpartition->is_auto_partition()) { - std::vector partition_keys = _vpartition->get_partition_keys(); - //TODO: use loop to create missing_vals for multi column. - CHECK(partition_keys.size() == 1) - << "now support only 1 partition column for auto partitions."; - auto partition_col = block->get_by_position(partition_keys[0]); - - std::vector missing_map; // indice of missing values in partition_col - missing_map.reserve(partition_col.column->size()); - - // try to find tablet and save missing value - for (int i = 0; i < num_rows; ++i) { - if (UNLIKELY(has_filtered_rows) && _block_convertor->filter_bitmap().Get(i)) { - continue; - } - const VOlapTablePartition* partition = nullptr; - bool is_continue = false; - uint32_t tablet_index = 0; - bool missing_this = false; - RETURN_IF_ERROR(_tablet_finder->find_tablet(state, block.get(), i, &partition, - tablet_index, stop_processing, - is_continue, &missing_this)); - if (missing_this) { - missing_map.push_back(i); - } else { - _generate_row_distribution_payload(channel_to_payload, partition, tablet_index, - i, 1); - } - } - missing_map.shrink_to_fit(); - - // for missing partition keys, calc the missing partition and save in _partitions_need_create - auto type = partition_col.type; - if (missing_map.size() > 0) { - auto return_type = part_func->data_type(); - - // expose the data column - vectorized::ColumnPtr range_left_col = block->get_by_position(result_idx).column; - if (auto* nullable = - check_and_get_column(*range_left_col)) { - range_left_col = nullable->get_nested_column_ptr(); - return_type = - assert_cast(return_type.get()) - ->get_nested_type(); - } - // calc the end value and save them. - _save_missing_values(range_left_col, return_type, missing_map); - // then call FE to create it. then FragmentExecutor will redo the load. - RETURN_IF_ERROR(_automatic_create_partition()); - // now we need to rollback the metrics - _number_input_rows -= rows; - state->update_num_rows_load_total(-rows); - state->update_num_bytes_load_total(-bytes); - DorisMetrics::instance()->load_rows->increment(-rows); - DorisMetrics::instance()->load_bytes->increment(-bytes); - // In the next round, we will _generate_row_distribution_payload again to get right payload of new tablet - LOG(INFO) << "Auto created partition. Send block again."; - return Status::NeedSendAgain(""); - } // creating done - } else { // not auto partition - for (int i = 0; i < num_rows; ++i) { - if (UNLIKELY(has_filtered_rows) && _block_convertor->filter_bitmap().Get(i)) { - continue; - } - const VOlapTablePartition* partition = nullptr; - bool is_continue = false; - uint32_t tablet_index = 0; - RETURN_IF_ERROR(_tablet_finder->find_tablet(state, block.get(), i, &partition, - tablet_index, stop_processing, - is_continue)); - if (is_continue) { - continue; - } - // each row - _generate_row_distribution_payload(channel_to_payload, partition, tablet_index, i, - 1); - } - } - } - _row_distribution_watch.stop(); - // Random distribution and the block belongs to a single tablet, we could optimize to append the whole - // block into node channel. - bool load_block_to_single_tablet = - !_vpartition->is_auto_partition() && _tablet_finder->is_single_tablet(); - if (load_block_to_single_tablet) { - SCOPED_RAW_TIMER(&_filter_ns); - // Filter block - if (has_filtered_rows) { - auto filter = vectorized::ColumnUInt8::create(block->rows(), 0); - vectorized::UInt8* filter_data = - static_cast(filter.get())->get_data().data(); - vectorized::IColumn::Filter& filter_col = - static_cast(filter.get())->get_data(); - for (size_t i = 0; i < filter_col.size(); ++i) { - filter_data[i] = !_block_convertor->filter_bitmap().Get(i); - } - RETURN_IF_CATCH_EXCEPTION(vectorized::Block::filter_block_internal( - block.get(), filter_col, block->columns())); - } - } - handle_block(input_block, rows, - _block_convertor->num_filtered_rows() + _tablet_finder->num_filtered_rows() - - filtered_rows); - // TODO: Before load, we need to projection unuseful column - // auto slots = _schema->tuple_desc()->slots(); - // for (auto desc : slots) { - // desc->col_pos(); - // } - // Add block to node channel - for (size_t i = 0; i < _channels.size(); i++) { - for (const auto& entry : channel_to_payload[i]) { - // if this node channel is already failed, this add_row will be skipped - auto st = entry.first->add_block( - block.get(), &entry.second, // entry.second is a [row -> tablet] mapping - // if it is load single tablet, then append this whole block - load_block_to_single_tablet); - if (!st.ok()) { - _channels[i]->mark_as_failed(entry.first, st.to_string()); - } - } - } - - // check intolerable failure - for (const auto& index_channel : _channels) { - RETURN_IF_ERROR(index_channel->check_intolerable_failure()); - } - return Status::OK(); -} - -Status VOlapTableSink::_cancel_channel_and_check_intolerable_failure( - Status status, const std::string& err_msg, const std::shared_ptr ich, - const std::shared_ptr nch) { - LOG(WARNING) << nch->channel_info() << ", close channel failed, err: " << err_msg; - ich->mark_as_failed(nch.get(), err_msg, -1); - // cancel the node channel in best effort - nch->cancel(err_msg); - - // check if index has intolerable failure - Status index_st = ich->check_intolerable_failure(); - if (!index_st.ok()) { - status = index_st; - } else if (Status st = ich->check_tablet_received_rows_consistency(); !st.ok()) { - status = st; - } else if (Status st = ich->check_tablet_filtered_rows_consistency(); !st.ok()) { - status = st; - } - return status; -} - -void VOlapTableSink::_cancel_all_channel(Status status) { - for (const auto& index_channel : _channels) { - index_channel->for_each_node_channel([&status](const std::shared_ptr& ch) { - ch->cancel(status.to_string()); - }); - } - LOG(INFO) << fmt::format( - "close olap table sink. load_id={}, txn_id={}, canceled all node channels due to " - "error: {}", - print_id(_load_id), _txn_id, status); -} - -Status VOlapTableSink::try_close(RuntimeState* state, Status exec_status) { - SCOPED_TIMER(_close_timer); - Status status = exec_status; - if (status.ok()) { - // only if status is ok can we call this _profile->total_time_counter(). - // if status is not ok, this sink may not be prepared, so that _profile is null - SCOPED_TIMER(_profile->total_time_counter()); - { - for (const auto& index_channel : _channels) { - if (!status.ok()) { - break; - } - index_channel->for_each_node_channel( - [this, &index_channel, &status](const std::shared_ptr& ch) { - if (!status.ok() || ch->is_closed()) { - return; - } - // only first try close, all node channels will mark_close() - ch->mark_close(); - if (ch->is_cancelled()) { - status = this->_cancel_channel_and_check_intolerable_failure( - status, ch->get_cancel_msg(), index_channel, ch); - } - }); - } // end for index channels - } - } - - if (!status.ok()) { - _cancel_all_channel(status); - _close_status = status; - _try_close = true; - } - - return Status::OK(); -} - -bool VOlapTableSink::is_close_done() { - // Only after try_close, need to wait rpc end. - if (!_try_close) { - return true; - } - bool close_done = true; - for (const auto& index_channel : _channels) { - index_channel->for_each_node_channel( - [&close_done](const std::shared_ptr& ch) { - close_done &= ch->is_send_data_rpc_done(); - }); - } - return close_done; -} - Status VOlapTableSink::close(RuntimeState* state, Status exec_status) { if (_closed) { return _close_status; } - if (!_prepare) { - DCHECK(!exec_status.ok()); - _cancel_all_channel(exec_status); - DataSink::close(state, exec_status); - _close_status = exec_status; - return _close_status; - } - - SCOPED_TIMER(_close_timer); - SCOPED_TIMER(_profile->total_time_counter()); - - try_close(state, exec_status); - - // If _close_status is not ok, all nodes have been canceled in try_close. - if (_close_status.ok()) { - auto status = Status::OK(); - // BE id -> add_batch method counter - std::unordered_map node_add_batch_counter_map; - int64_t serialize_batch_ns = 0, queue_push_lock_ns = 0, actual_consume_ns = 0, - total_add_batch_exec_time_ns = 0, max_add_batch_exec_time_ns = 0, - total_wait_exec_time_ns = 0, max_wait_exec_time_ns = 0, total_add_batch_num = 0, - num_node_channels = 0; - VNodeChannelStat channel_stat; - - for (const auto& index_channel : _channels) { - if (!status.ok()) { - break; - } - int64_t add_batch_exec_time = 0; - int64_t wait_exec_time = 0; - index_channel->for_each_node_channel( - [this, &index_channel, &status, &state, &node_add_batch_counter_map, - &serialize_batch_ns, &channel_stat, &queue_push_lock_ns, &actual_consume_ns, - &total_add_batch_exec_time_ns, &add_batch_exec_time, &total_wait_exec_time_ns, - &wait_exec_time, - &total_add_batch_num](const std::shared_ptr& ch) { - if (!status.ok() || ch->is_closed()) { - return; - } - // in pipeline, all node channels are done or canceled, will not block. - // no pipeline, close may block waiting. - auto s = ch->close_wait(state); - if (!s.ok()) { - status = this->_cancel_channel_and_check_intolerable_failure( - status, s.to_string(), index_channel, ch); - } - ch->time_report(&node_add_batch_counter_map, &serialize_batch_ns, - &channel_stat, &queue_push_lock_ns, &actual_consume_ns, - &total_add_batch_exec_time_ns, &add_batch_exec_time, - &total_wait_exec_time_ns, &wait_exec_time, - &total_add_batch_num); - }); - - // Due to the non-determinism of compaction, the rowsets of each replica may be different from each other on different - // BE nodes. The number of rows filtered in SegmentWriter depends on the historical rowsets located in the correspoding - // BE node. So we check the number of rows filtered on each succeccful BE to ensure the consistency of the current load - if (status.ok() && !_write_single_replica && _schema->is_strict_mode() && - _schema->is_partial_update()) { - if (Status st = index_channel->check_tablet_filtered_rows_consistency(); !st.ok()) { - status = st; - } else { - state->set_num_rows_filtered_in_strict_mode_partial_update( - index_channel->num_rows_filtered()); - } - } - - num_node_channels += index_channel->num_node_channels(); - if (add_batch_exec_time > max_add_batch_exec_time_ns) { - max_add_batch_exec_time_ns = add_batch_exec_time; - } - if (wait_exec_time > max_wait_exec_time_ns) { - max_wait_exec_time_ns = wait_exec_time; - } - } // end for index channels - - if (status.ok()) { - // TODO need to be improved - LOG(INFO) << "total mem_exceeded_block_ns=" << channel_stat.mem_exceeded_block_ns - << ", total queue_push_lock_ns=" << queue_push_lock_ns - << ", total actual_consume_ns=" << actual_consume_ns - << ", load id=" << print_id(_load_id); - - COUNTER_SET(_input_rows_counter, _number_input_rows); - COUNTER_SET(_output_rows_counter, _number_output_rows); - COUNTER_SET(_filtered_rows_counter, - _block_convertor->num_filtered_rows() + - _tablet_finder->num_filtered_rows() + - state->num_rows_filtered_in_strict_mode_partial_update()); - COUNTER_SET(_send_data_timer, _send_data_ns); - COUNTER_SET(_row_distribution_timer, (int64_t)_row_distribution_watch.elapsed_time()); - COUNTER_SET(_filter_timer, _filter_ns); - COUNTER_SET(_append_node_channel_timer, channel_stat.append_node_channel_ns); - COUNTER_SET(_where_clause_timer, channel_stat.where_clause_ns); - COUNTER_SET(_wait_mem_limit_timer, channel_stat.mem_exceeded_block_ns); - COUNTER_SET(_validate_data_timer, _block_convertor->validate_data_ns()); - COUNTER_SET(_serialize_batch_timer, serialize_batch_ns); - COUNTER_SET(_non_blocking_send_work_timer, actual_consume_ns); - COUNTER_SET(_total_add_batch_exec_timer, total_add_batch_exec_time_ns); - COUNTER_SET(_max_add_batch_exec_timer, max_add_batch_exec_time_ns); - COUNTER_SET(_total_wait_exec_timer, total_wait_exec_time_ns); - COUNTER_SET(_max_wait_exec_timer, max_wait_exec_time_ns); - COUNTER_SET(_add_batch_number, total_add_batch_num); - COUNTER_SET(_num_node_channels, num_node_channels); - // _number_input_rows don't contain num_rows_load_filtered and num_rows_load_unselected in scan node - int64_t num_rows_load_total = _number_input_rows + state->num_rows_load_filtered() + - state->num_rows_load_unselected(); - state->set_num_rows_load_total(num_rows_load_total); - state->update_num_rows_load_filtered( - _block_convertor->num_filtered_rows() + _tablet_finder->num_filtered_rows() + - state->num_rows_filtered_in_strict_mode_partial_update()); - state->update_num_rows_load_unselected( - _tablet_finder->num_immutable_partition_filtered_rows()); - - // print log of add batch time of all node, for tracing load performance easily - std::stringstream ss; - ss << "finished to close olap table sink. load_id=" << print_id(_load_id) - << ", txn_id=" << _txn_id - << ", node add batch time(ms)/wait execution time(ms)/close time(ms)/num: "; - for (auto const& pair : node_add_batch_counter_map) { - ss << "{" << pair.first << ":(" << (pair.second.add_batch_execution_time_us / 1000) - << ")(" << (pair.second.add_batch_wait_execution_time_us / 1000) << ")(" - << pair.second.close_wait_time_ms << ")(" << pair.second.add_batch_num << ")} "; - } - LOG(INFO) << ss.str(); - } else { - _cancel_all_channel(status); - } - _close_status = status; - } - - // Sender join() must put after node channels mark_close/cancel. - // But there is no specific sequence required between sender join() & close_wait(). - if (_sender_thread) { - bthread_join(_sender_thread, nullptr); - // We have to wait all task in _send_batch_thread_pool_token finished, - // because it is difficult to handle concurrent problem if we just - // shutdown it. - _send_batch_thread_pool_token->wait(); - } - - DataSink::close(state, exec_status); + _close_status = AsyncWriterSink::close(state, exec_status); return _close_status; } -} // namespace stream_load +} // namespace vectorized } // namespace doris diff --git a/be/src/vec/sink/vtablet_sink.h b/be/src/vec/sink/vtablet_sink.h index 2720ae53bc358d..a5208f070cd9b7 100644 --- a/be/src/vec/sink/vtablet_sink.h +++ b/be/src/vec/sink/vtablet_sink.h @@ -71,511 +71,32 @@ #include "vec/core/block.h" #include "vec/data_types/data_type.h" #include "vec/exprs/vexpr_fwd.h" +#include "vec/sink/async_writer_sink.h" +#include "vec/sink/writer/vtablet_writer.h" -namespace doris { -class ObjectPool; -class RowDescriptor; -class RuntimeState; -class TDataSink; -class TExpr; -class Thread; -class ThreadPoolToken; -class TupleDescriptor; -template -class RefCountClosure; - -namespace stream_load { - -class OlapTableBlockConvertor; -class OlapTabletFinder; - -// The counter of add_batch rpc of a single node -struct AddBatchCounter { - // total execution time of a add_batch rpc - int64_t add_batch_execution_time_us = 0; - // lock waiting time in a add_batch rpc - int64_t add_batch_wait_execution_time_us = 0; - // number of add_batch call - int64_t add_batch_num = 0; - // time passed between marked close and finish close - int64_t close_wait_time_ms = 0; - - AddBatchCounter& operator+=(const AddBatchCounter& rhs) { - add_batch_execution_time_us += rhs.add_batch_execution_time_us; - add_batch_wait_execution_time_us += rhs.add_batch_wait_execution_time_us; - add_batch_num += rhs.add_batch_num; - close_wait_time_ms += rhs.close_wait_time_ms; - return *this; - } - friend AddBatchCounter operator+(const AddBatchCounter& lhs, const AddBatchCounter& rhs) { - AddBatchCounter sum = lhs; - sum += rhs; - return sum; - } -}; - -// It's very error-prone to guarantee the handler capture vars' & this closure's destruct sequence. -// So using create() to get the closure pointer is recommended. We can delete the closure ptr before the capture vars destruction. -// Delete this point is safe, don't worry about RPC callback will run after ReusableClosure deleted. -// "Ping-Pong" between sender and receiver, `try_set_in_flight` when send, `clear_in_flight` after rpc failure or callback, -// then next send will start, and it will wait for the rpc callback to complete when it is destroyed. -template -class ReusableClosure final : public google::protobuf::Closure { -public: - ReusableClosure() : cid(INVALID_BTHREAD_ID) {} - ~ReusableClosure() override { - // shouldn't delete when Run() is calling or going to be called, wait for current Run() done. - join(); - SCOPED_TRACK_MEMORY_TO_UNKNOWN(); - cntl.Reset(); - } - - static ReusableClosure* create() { return new ReusableClosure(); } - - void addFailedHandler(const std::function& fn) { failed_handler = fn; } - void addSuccessHandler(const std::function& fn) { success_handler = fn; } - - void join() { - // We rely on in_flight to assure one rpc is running, - // while cid is not reliable due to memory order. - // in_flight is written before getting callid, - // so we can not use memory fence to synchronize. - while (_packet_in_flight) { - // cid here is complicated - if (cid != INVALID_BTHREAD_ID) { - // actually cid may be the last rpc call id. - brpc::Join(cid); - } - if (_packet_in_flight) { - std::this_thread::sleep_for(std::chrono::milliseconds(10)); - } - } - } - - // plz follow this order: reset() -> set_in_flight() -> send brpc batch - void reset() { - SCOPED_TRACK_MEMORY_TO_UNKNOWN(); - cntl.Reset(); - cid = cntl.call_id(); - } - - // if _packet_in_flight == false, set it to true. Return true. - // if _packet_in_flight == true, Return false. - bool try_set_in_flight() { - bool value = false; - return _packet_in_flight.compare_exchange_strong(value, true); - } - - void clear_in_flight() { _packet_in_flight = false; } - - bool is_packet_in_flight() { return _packet_in_flight; } - - void end_mark() { - DCHECK(_is_last_rpc == false); - _is_last_rpc = true; - } - - void Run() override { - DCHECK(_packet_in_flight); - if (cntl.Failed()) { - LOG(WARNING) << "failed to send brpc batch, error=" << berror(cntl.ErrorCode()) - << ", error_text=" << cntl.ErrorText(); - failed_handler(_is_last_rpc); - } else { - success_handler(result, _is_last_rpc); - } - clear_in_flight(); - } - - brpc::Controller cntl; - T result; - -private: - brpc::CallId cid; - std::atomic _packet_in_flight {false}; - std::atomic _is_last_rpc {false}; - std::function failed_handler; - std::function success_handler; -}; - -class IndexChannel; -class VOlapTableSink; - -// pair -using Payload = std::pair, std::vector>; - -class VNodeChannelStat { -public: - VNodeChannelStat& operator+=(const VNodeChannelStat& stat) { - mem_exceeded_block_ns += stat.mem_exceeded_block_ns; - where_clause_ns += stat.where_clause_ns; - append_node_channel_ns += stat.append_node_channel_ns; - return *this; - }; - - int64_t mem_exceeded_block_ns = 0; - int64_t where_clause_ns = 0; - int64_t append_node_channel_ns = 0; -}; - -// every NodeChannel keeps a data transmission channel with one BE. for multiple times open, it has a dozen of requests and corresponding closures. -class VNodeChannel { -public: - VNodeChannel(VOlapTableSink* parent, IndexChannel* index_channel, int64_t node_id, - bool is_incremental = false); - - ~VNodeChannel(); - - // called before open, used to add tablet located in this backend. called by IndexChannel::init - void add_tablet(const TTabletWithPartition& tablet) { _all_tablets.emplace_back(tablet); } - std::string debug_tablets() const { - std::stringstream ss; - for (auto& tab : _all_tablets) { - tab.printTo(ss); - ss << '\n'; - } - return ss.str(); - } - - void add_slave_tablet_nodes(int64_t tablet_id, const std::vector& slave_nodes) { - _slave_tablet_nodes[tablet_id] = slave_nodes; - } - - // build a request and build corresponding connect to BE. - void open(); - // for auto partition, we use this to open more tablet. - void incremental_open(); - - Status init(RuntimeState* state); - - // this will block until all request transmission which were opened or incremental opened finished. - Status open_wait(); - - Status add_block(vectorized::Block* block, const Payload* payload, bool is_append = false); - - // @return: unfinished running channels. - // @caller: VOlapTabletSink::_send_batch_process. it's a continual asynchronous process. - int try_send_and_fetch_status(RuntimeState* state, - std::unique_ptr& thread_pool_token); - // when there's pending block found by try_send_and_fetch_status(), we will awake a thread to send it. - void try_send_pending_block(RuntimeState* state); - - void clear_all_blocks(); - - // two ways to stop channel: - // 1. mark_close()->close_wait() PS. close_wait() will block waiting for the last AddBatch rpc response. - // 2. just cancel() - void mark_close(); - - bool is_send_data_rpc_done() const; - - bool is_closed() const { return _is_closed; } - bool is_cancelled() const { return _cancelled; } - std::string get_cancel_msg() { - std::stringstream ss; - ss << "close wait failed coz rpc error"; - { - std::lock_guard l(_cancel_msg_lock); - if (_cancel_msg != "") { - ss << ". " << _cancel_msg; - } - } - return ss.str(); - } - - // two ways to stop channel: - // 1. mark_close()->close_wait() PS. close_wait() will block waiting for the last AddBatch rpc response. - // 2. just cancel() - Status close_wait(RuntimeState* state); - - void cancel(const std::string& cancel_msg); - - void time_report(std::unordered_map* add_batch_counter_map, - int64_t* serialize_batch_ns, VNodeChannelStat* stat, - int64_t* queue_push_lock_ns, int64_t* actual_consume_ns, - int64_t* total_add_batch_exec_time_ns, int64_t* add_batch_exec_time_ns, - int64_t* total_wait_exec_time_ns, int64_t* wait_exec_time_ns, - int64_t* total_add_batch_num) const { - (*add_batch_counter_map)[_node_id] += _add_batch_counter; - (*add_batch_counter_map)[_node_id].close_wait_time_ms = _close_time_ms; - *serialize_batch_ns += _serialize_batch_ns; - *stat += _stat; - *queue_push_lock_ns += _queue_push_lock_ns; - *actual_consume_ns += _actual_consume_ns; - *add_batch_exec_time_ns = (_add_batch_counter.add_batch_execution_time_us * 1000); - *total_add_batch_exec_time_ns += *add_batch_exec_time_ns; - *wait_exec_time_ns = (_add_batch_counter.add_batch_wait_execution_time_us * 1000); - *total_wait_exec_time_ns += *wait_exec_time_ns; - *total_add_batch_num += _add_batch_counter.add_batch_num; - } - - int64_t node_id() const { return _node_id; } - std::string host() const { return _node_info.host; } - std::string name() const { return _name; } - - Status none_of(std::initializer_list vars); - - std::string channel_info() const { - return fmt::format("{}, {}, node={}:{}", _name, _load_info, _node_info.host, - _node_info.brpc_port); - } - - size_t get_pending_bytes() { return _pending_batches_bytes; } - - bool is_incremental() const { return _is_incremental; } - -protected: - // make a real open request for relative BE's load channel. - void _open_internal(bool is_incremental); - - void _close_check(); - void _cancel_with_msg(const std::string& msg); - - void _add_block_success_callback(const PTabletWriterAddBlockResult& result, bool is_last_rpc); - void _add_block_failed_callback(bool is_last_rpc); - - VOlapTableSink* _parent = nullptr; - IndexChannel* _index_channel = nullptr; - int64_t _node_id = -1; - std::string _load_info; - std::string _name; - - std::shared_ptr _node_channel_tracker; - - TupleDescriptor* _tuple_desc = nullptr; - NodeInfo _node_info; - - // this should be set in init() using config - int _rpc_timeout_ms = 60000; - int64_t _next_packet_seq = 0; - MonotonicStopWatch _timeout_watch; - - // the timestamp when this node channel be marked closed and finished closed - uint64_t _close_time_ms = 0; - - // user cancel or get some errors - std::atomic _cancelled {false}; - doris::SpinLock _cancel_msg_lock; - std::string _cancel_msg; - - // send finished means the consumer thread which send the rpc can exit - std::atomic _send_finished {false}; - - // add batches finished means the last rpc has be response, used to check whether this channel can be closed - std::atomic _add_batches_finished {false}; // reuse for vectorized - - bool _eos_is_produced {false}; // only for restricting producer behaviors - - std::unique_ptr _row_desc; - int _batch_size = 0; - - // limit _pending_batches size - std::atomic _pending_batches_bytes {0}; - size_t _max_pending_batches_bytes {(size_t)config::nodechannel_pending_queue_max_bytes}; - std::mutex _pending_batches_lock; // reuse for vectorized - std::atomic _pending_batches_num {0}; // reuse for vectorized - - std::shared_ptr _stub = nullptr; - // because we have incremantal open, we should keep one relative closure for one request. it's similarly for adding block. - std::vector*> _open_closures; - - std::vector _all_tablets; - // map from tablet_id to node_id where slave replicas locate in - std::unordered_map> _slave_tablet_nodes; - std::vector _tablet_commit_infos; - - AddBatchCounter _add_batch_counter; - std::atomic _serialize_batch_ns {0}; - std::atomic _queue_push_lock_ns {0}; - std::atomic _actual_consume_ns {0}; - - VNodeChannelStat _stat; - // lock to protect _is_closed. - // The methods in the IndexChannel are called back in the RpcClosure in the NodeChannel. - // However, this rpc callback may occur after the whole task is finished (e.g. due to network latency), - // and by that time the IndexChannel may have been destructured, so we should not call the - // IndexChannel methods anymore, otherwise the BE will crash. - // Therefore, we use the _is_closed and _closed_lock to ensure that the RPC callback - // function will not call the IndexChannel method after the NodeChannel is closed. - // The IndexChannel is definitely accessible until the NodeChannel is closed. - std::mutex _closed_lock; - bool _is_closed = false; - - RuntimeState* _state; - // rows number received per tablet, tablet_id -> rows_num - std::vector> _tablets_received_rows; - // rows number filtered per tablet, tablet_id -> filtered_rows_num - std::vector> _tablets_filtered_rows; - - // build a _cur_mutable_block and push into _pending_blocks. when not building, this block is empty. - std::unique_ptr _cur_mutable_block; - PTabletWriterAddBlockRequest _cur_add_block_request; - - using AddBlockReq = - std::pair, PTabletWriterAddBlockRequest>; - std::queue _pending_blocks; - ReusableClosure* _add_block_closure = nullptr; - - bool _is_incremental; -}; +namespace doris::vectorized { +inline constexpr char VOLAP_TABLE_SINK[] = "VOlapTableSink"; // Write block data to Olap Table. // When OlapTableSink::open() called, there will be a consumer thread running in the background. // When you call VOlapTableSink::send(), you will be the producer who products pending batches. // Join the consumer thread in close(). -class VOlapTableSink : public DataSink { +class VOlapTableSink final : public AsyncWriterSink { public: // Construct from thrift struct which is generated by FE. VOlapTableSink(ObjectPool* pool, const RowDescriptor& row_desc, - const std::vector& texprs, Status* status); - - ~VOlapTableSink() override; + const std::vector& texprs, bool group_commit); Status init(const TDataSink& sink) override; - // TODO: unify the code of prepare/open/close with result sink - Status prepare(RuntimeState* state) override; - - Status open(RuntimeState* state) override; - - Status try_close(RuntimeState* state, Status exec_status) override; - // if true, all node channels rpc done, can start close(). - bool is_close_done() override; - Status close(RuntimeState* state, Status close_status) override; - Status send(RuntimeState* state, vectorized::Block* block, bool eos = false) override; - - size_t get_pending_bytes() const; - - // the consumer func of sending pending batches in every NodeChannel. - // use polling & NodeChannel::try_send_and_fetch_status() to achieve nonblocking sending. - // only focus on pending batches and channel status, the internal errors of NodeChannels will be handled by the producer - void _send_batch_process(); - // handle block after data is filtered, only useful for GroupCommitVOlapTabletSink - virtual void handle_block(vectorized::Block* input_block, int64_t rows, int64_t filter_rows) {} + Status close(RuntimeState* state, Status exec_status) override; private: - friend class VNodeChannel; - friend class IndexChannel; - - using ChannelDistributionPayload = std::vector>; - - // payload for each row - void _generate_row_distribution_payload(ChannelDistributionPayload& payload, - const VOlapTablePartition* partition, - uint32_t tablet_index, int row_idx, size_t row_cnt); - Status _single_partition_generate(RuntimeState* state, vectorized::Block* block, - ChannelDistributionPayload& channel_to_payload, - size_t num_rows, bool has_filtered_rows); - - Status _cancel_channel_and_check_intolerable_failure(Status status, const std::string& err_msg, - const std::shared_ptr ich, - const std::shared_ptr nch); - - void _cancel_all_channel(Status status); - - std::pair _get_partition_function(); - - void _save_missing_values(vectorized::ColumnPtr col, vectorized::DataTypePtr value_type, - std::vector filter); - - // create partitions when need for auto-partition table using #_partitions_need_create. - Status _automatic_create_partition(); - - Status _incremental_open_node_channel(const std::vector& partitions); - - std::shared_ptr _mem_tracker; - ObjectPool* _pool; - // unique load id - PUniqueId _load_id; - int64_t _txn_id = -1; - int _num_replicas = -1; - int _tuple_desc_id = -1; - - // this is tuple descriptor of destination OLAP table - TupleDescriptor* _output_tuple_desc = nullptr; - RowDescriptor* _output_row_desc = nullptr; - - // number of senders used to insert into OlapTable, if we only support single node insert, - // all data from select should collectted and then send to OlapTable. - // To support multiple senders, we maintain a channel for each sender. - int _sender_id = -1; - int _num_senders = -1; - bool _is_high_priority = false; - - // TODO(zc): think about cache this data - std::shared_ptr _schema; - OlapTableLocationParam* _location = nullptr; - bool _write_single_replica = false; - OlapTableLocationParam* _slave_location = nullptr; - DorisNodesInfo* _nodes_info = nullptr; - - std::unique_ptr _tablet_finder; - - // index_channel - std::mutex _stop_check_channel; - std::vector> _channels; - std::unordered_map> _index_id_to_channel; - - bthread_t _sender_thread = 0; - std::unique_ptr _send_batch_thread_pool_token; - - // support only one partition column now - std::vector> _partitions_need_create; - - std::unique_ptr _block_convertor; - // Stats for this - int64_t _send_data_ns = 0; - int64_t _number_input_rows = 0; - int64_t _number_output_rows = 0; - int64_t _filter_ns = 0; - - MonotonicStopWatch _row_distribution_watch; - - RuntimeProfile::Counter* _input_rows_counter = nullptr; - RuntimeProfile::Counter* _output_rows_counter = nullptr; - RuntimeProfile::Counter* _filtered_rows_counter = nullptr; - RuntimeProfile::Counter* _send_data_timer = nullptr; - RuntimeProfile::Counter* _row_distribution_timer = nullptr; - RuntimeProfile::Counter* _append_node_channel_timer = nullptr; - RuntimeProfile::Counter* _filter_timer = nullptr; - RuntimeProfile::Counter* _where_clause_timer = nullptr; - RuntimeProfile::Counter* _add_partition_request_timer = nullptr; - RuntimeProfile::Counter* _wait_mem_limit_timer = nullptr; - RuntimeProfile::Counter* _validate_data_timer = nullptr; - RuntimeProfile::Counter* _open_timer = nullptr; - RuntimeProfile::Counter* _close_timer = nullptr; - RuntimeProfile::Counter* _non_blocking_send_timer = nullptr; - RuntimeProfile::Counter* _non_blocking_send_work_timer = nullptr; - RuntimeProfile::Counter* _serialize_batch_timer = nullptr; - RuntimeProfile::Counter* _total_add_batch_exec_timer = nullptr; - RuntimeProfile::Counter* _max_add_batch_exec_timer = nullptr; - RuntimeProfile::Counter* _total_wait_exec_timer = nullptr; - RuntimeProfile::Counter* _max_wait_exec_timer = nullptr; - RuntimeProfile::Counter* _add_batch_number = nullptr; - RuntimeProfile::Counter* _num_node_channels = nullptr; - - // load mem limit is for remote load channel - int64_t _load_mem_limit = -1; - - // the timeout of load channels opened by this tablet sink. in second - int64_t _load_channel_timeout_s = 0; - - int32_t _send_batch_parallelism = 1; - // Save the status of try_close() and close() method - Status _close_status; - bool _try_close = false; - bool _prepare = false; - - // User can change this config at runtime, avoid it being modified during query or loading process. - bool _transfer_large_data_by_brpc = false; - - VOlapTablePartitionParam* _vpartition = nullptr; - vectorized::VExprContextSPtrs _output_vexpr_ctxs; + bool _group_commit = false; - RuntimeState* _state = nullptr; + Status _close_status = Status::OK(); }; -} // namespace stream_load -} // namespace doris +} // namespace doris::vectorized diff --git a/be/src/vec/sink/vtablet_sink_v2.cpp b/be/src/vec/sink/vtablet_sink_v2.cpp index eed24133ade6c3..a251427c44f3d8 100644 --- a/be/src/vec/sink/vtablet_sink_v2.cpp +++ b/be/src/vec/sink/vtablet_sink_v2.cpp @@ -64,7 +64,7 @@ namespace doris { class TExpr; -namespace stream_load { +namespace vectorized { VOlapTableSinkV2::VOlapTableSinkV2(ObjectPool* pool, const RowDescriptor& row_desc, const std::vector& texprs, Status* status) @@ -283,7 +283,7 @@ Status VOlapTableSinkV2::send(RuntimeState* state, vectorized::Block* input_bloc std::shared_ptr block; bool has_filtered_rows = false; RETURN_IF_ERROR(_block_convertor->validate_and_convert_block( - state, input_block, block, _output_vexpr_ctxs, input_rows, eos, has_filtered_rows)); + state, input_block, block, _output_vexpr_ctxs, input_rows, has_filtered_rows)); // clear and release the references of columns input_block->clear(); @@ -458,5 +458,5 @@ Status VOlapTableSinkV2::_close_load(const Streams& streams) { return Status::OK(); } -} // namespace stream_load +} // namespace vectorized } // namespace doris diff --git a/be/src/vec/sink/vtablet_sink_v2.h b/be/src/vec/sink/vtablet_sink_v2.h index 9aa4877cbe22a1..f70f74b9da649b 100644 --- a/be/src/vec/sink/vtablet_sink_v2.h +++ b/be/src/vec/sink/vtablet_sink_v2.h @@ -75,7 +75,7 @@ class TExpr; class TabletSchema; class TupleDescriptor; -namespace stream_load { +namespace vectorized { class OlapTableBlockConvertor; class OlapTabletFinder; @@ -226,5 +226,5 @@ class VOlapTableSinkV2 final : public DataSink { friend class StreamSinkHandler; }; -} // namespace stream_load +} // namespace vectorized } // namespace doris diff --git a/be/src/vec/sink/writer/async_result_writer.cpp b/be/src/vec/sink/writer/async_result_writer.cpp index b30abfb46b52eb..8d29c1441d303a 100644 --- a/be/src/vec/sink/writer/async_result_writer.cpp +++ b/be/src/vec/sink/writer/async_result_writer.cpp @@ -75,17 +75,20 @@ void AsyncResultWriter::start_writer(RuntimeState* state, RuntimeProfile* profil } void AsyncResultWriter::process_block(RuntimeState* state, RuntimeProfile* profile) { - _writer_status = open(state, profile); + if (auto status = open(state, profile); !status.ok()) { + force_close(status); + } + if (_writer_status.ok()) { while (true) { { std::unique_lock l(_m); - while (!_eos && _data_queue.empty() && !_force_close) { + while (!_eos && _data_queue.empty() && _writer_status.ok()) { _cv.wait(l); } } - if ((_eos && _data_queue.empty()) || _force_close) { + if ((_eos && _data_queue.empty()) || !_writer_status.ok()) { _data_queue.clear(); break; } @@ -101,8 +104,8 @@ void AsyncResultWriter::process_block(RuntimeState* state, RuntimeProfile* profi // if not in transaction or status is in error or force close we can do close in // async IO thread - if (!_writer_status.ok() || _force_close || !in_transaction()) { - close(); + if (!_writer_status.ok() || !in_transaction()) { + close(_writer_status); _need_normal_close = false; } _writer_thread_closed = true; @@ -120,9 +123,9 @@ Status AsyncResultWriter::_projection_block(doris::vectorized::Block& input_bloc return status; } -void AsyncResultWriter::force_close() { +void AsyncResultWriter::force_close(Status s) { std::lock_guard l(_m); - _force_close = true; + _writer_status = s; _cv.notify_one(); } diff --git a/be/src/vec/sink/writer/async_result_writer.h b/be/src/vec/sink/writer/async_result_writer.h index 2f70b05266b74f..9806d20945c1b3 100644 --- a/be/src/vec/sink/writer/async_result_writer.h +++ b/be/src/vec/sink/writer/async_result_writer.h @@ -48,9 +48,7 @@ class AsyncResultWriter : public ResultWriter { public: AsyncResultWriter(const VExprContextSPtrs& output_expr_ctxs); - Status close() override { return Status::OK(); } - - void force_close(); + void force_close(Status s); virtual bool in_transaction() { return false; } @@ -92,7 +90,6 @@ class AsyncResultWriter : public ResultWriter { std::deque> _data_queue; Status _writer_status = Status::OK(); bool _eos = false; - bool _force_close = false; bool _need_normal_close = true; bool _writer_thread_closed = false; }; diff --git a/be/src/vec/sink/writer/vfile_result_writer.cpp b/be/src/vec/sink/writer/vfile_result_writer.cpp index 68b8d1c1ea2c54..bb67f635b2f857 100644 --- a/be/src/vec/sink/writer/vfile_result_writer.cpp +++ b/be/src/vec/sink/writer/vfile_result_writer.cpp @@ -46,9 +46,7 @@ #include "util/mysql_row_buffer.h" #include "util/s3_uri.h" #include "util/s3_util.h" -#include "util/types.h" #include "util/uid_util.h" -#include "vec/columns/column.h" #include "vec/columns/column_string.h" #include "vec/columns/column_vector.h" #include "vec/core/block.h" @@ -81,7 +79,7 @@ VFileResultWriter::VFileResultWriter(const ResultFileOptions* file_opts, _output_object_data = output_object_data; } -Status VFileResultWriter::_init(RuntimeState* state, RuntimeProfile* profile) { +Status VFileResultWriter::open(RuntimeState* state, RuntimeProfile* profile) { _state = state; _init_profile(profile); // Delete existing files @@ -418,7 +416,7 @@ Status VFileResultWriter::_delete_dir() { return Status::OK(); } -Status VFileResultWriter::close() { +Status VFileResultWriter::close(Status) { // the following 2 profile "_written_rows_counter" and "_writer_close_timer" // must be outside the `_close_file_writer()`. // because `_close_file_writer()` may be called in deconstructor, diff --git a/be/src/vec/sink/writer/vfile_result_writer.h b/be/src/vec/sink/writer/vfile_result_writer.h index 69a26714dc64d2..b328eac76b82c3 100644 --- a/be/src/vec/sink/writer/vfile_result_writer.h +++ b/be/src/vec/sink/writer/vfile_result_writer.h @@ -60,11 +60,9 @@ class VFileResultWriter final : public AsyncResultWriter { Status append_block(Block& block) override; - Status close() override; + Status close(Status s = Status::OK()) override; - Status open(RuntimeState* state, RuntimeProfile* profile) override { - return _init(state, profile); - } + Status open(RuntimeState* state, RuntimeProfile* profile) override; // file result writer always return statistic result in one row int64_t get_written_rows() const override { return 1; } @@ -75,7 +73,6 @@ class VFileResultWriter final : public AsyncResultWriter { } private: - Status _init(RuntimeState* state, RuntimeProfile*); Status _write_file(const Block& block); void _init_profile(RuntimeProfile*); diff --git a/be/src/vec/sink/writer/vjdbc_table_writer.h b/be/src/vec/sink/writer/vjdbc_table_writer.h index 205f0835f5dbf6..b1e4d5ad284f4d 100644 --- a/be/src/vec/sink/writer/vjdbc_table_writer.h +++ b/be/src/vec/sink/writer/vjdbc_table_writer.h @@ -46,7 +46,7 @@ class VJdbcTableWriter final : public AsyncResultWriter, public JdbcConnector { Status append_block(vectorized::Block& block) override; - Status close() override { return JdbcConnector::close(); } + Status close(Status s) override { return JdbcConnector::close(s); } bool in_transaction() override { return TableConnector::_is_in_transaction; } diff --git a/be/src/vec/sink/writer/vmysql_table_writer.cpp b/be/src/vec/sink/writer/vmysql_table_writer.cpp index 6897a6b3e24b65..77e054e05c9c4a 100644 --- a/be/src/vec/sink/writer/vmysql_table_writer.cpp +++ b/be/src/vec/sink/writer/vmysql_table_writer.cpp @@ -74,7 +74,7 @@ VMysqlTableWriter::VMysqlTableWriter(const TDataSink& t_sink, _conn_info.charset = t_mysql_sink.charset; } -Status VMysqlTableWriter::close() { +Status VMysqlTableWriter::close(Status) { if (_mysql_conn) { mysql_close(_mysql_conn); } diff --git a/be/src/vec/sink/writer/vmysql_table_writer.h b/be/src/vec/sink/writer/vmysql_table_writer.h index 9028801144ddeb..b3cd65057c7d2f 100644 --- a/be/src/vec/sink/writer/vmysql_table_writer.h +++ b/be/src/vec/sink/writer/vmysql_table_writer.h @@ -53,7 +53,7 @@ class VMysqlTableWriter final : public AsyncResultWriter { Status append_block(vectorized::Block& block) override; - Status close() override; + Status close(Status) override; private: Status _insert_row(vectorized::Block& block, size_t row); diff --git a/be/src/vec/sink/writer/vodbc_table_writer.h b/be/src/vec/sink/writer/vodbc_table_writer.h index 3df973e4b1253f..4c0e6a19f6ab22 100644 --- a/be/src/vec/sink/writer/vodbc_table_writer.h +++ b/be/src/vec/sink/writer/vodbc_table_writer.h @@ -46,7 +46,7 @@ class VOdbcTableWriter final : public AsyncResultWriter, public ODBCConnector { Status append_block(vectorized::Block& block) override; - Status close() override { return ODBCConnector::close(); } + Status close(Status s) override { return ODBCConnector::close(s); } bool in_transaction() override { return TableConnector::_is_in_transaction; } diff --git a/be/src/vec/sink/writer/vtablet_writer.cpp b/be/src/vec/sink/writer/vtablet_writer.cpp new file mode 100644 index 00000000000000..322451cde37a87 --- /dev/null +++ b/be/src/vec/sink/writer/vtablet_writer.cpp @@ -0,0 +1,1819 @@ +// 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 "vtablet_writer.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include "util/runtime_profile.h" +#include "vec/data_types/data_type.h" +#include "vec/exprs/vexpr_fwd.h" +#include "vec/runtime/vdatetime_value.h" +#include "vec/sink/vtablet_sink.h" + +#ifdef DEBUG +#include +#endif + +// IWYU pragma: no_include +#include "common/compiler_util.h" // IWYU pragma: keep +#include "common/logging.h" +#include "common/object_pool.h" +#include "common/status.h" +#include "exec/tablet_info.h" +#include "runtime/client_cache.h" +#include "runtime/define_primitive_type.h" +#include "runtime/descriptors.h" +#include "runtime/exec_env.h" +#include "runtime/runtime_state.h" +#include "runtime/thread_context.h" +#include "service/backend_options.h" +#include "util/defer_op.h" +#include "util/doris_metrics.h" +#include "util/network_util.h" +#include "util/proto_util.h" +#include "util/ref_count_closure.h" +#include "util/telemetry/telemetry.h" +#include "util/threadpool.h" +#include "util/thrift_rpc_helper.h" +#include "util/thrift_util.h" +#include "util/time.h" +#include "util/uid_util.h" +#include "vec/columns/column.h" +#include "vec/columns/column_const.h" +#include "vec/columns/column_decimal.h" +#include "vec/columns/column_nullable.h" +#include "vec/columns/column_vector.h" +#include "vec/columns/columns_number.h" +#include "vec/common/assert_cast.h" +#include "vec/core/block.h" +#include "vec/core/future_block.h" +#include "vec/core/types.h" +#include "vec/data_types/data_type_nullable.h" +#include "vec/exprs/vexpr.h" +#include "vec/sink/vtablet_block_convertor.h" +#include "vec/sink/vtablet_finder.h" + +namespace doris { +class TExpr; + +namespace vectorized { + +Status IndexChannel::init(RuntimeState* state, const std::vector& tablets) { + SCOPED_CONSUME_MEM_TRACKER(_index_channel_tracker.get()); + for (auto& tablet : tablets) { + // First find the location BEs of this tablet + auto tablet_locations = _parent->_location->find_tablet(tablet.tablet_id); + if (tablet_locations == nullptr) { + return Status::InternalError("unknown tablet, tablet_id={}", tablet.tablet_id); + } + std::vector> channels; + // For tablet, deal with its' all replica (in some node). + for (auto& replica_node_id : tablet_locations->node_ids) { + std::shared_ptr channel; + auto it = _node_channels.find(replica_node_id); + // when we prepare for TableSink or incremental open tablet, we need init + if (it == _node_channels.end()) { + // NodeChannel is not added to the _parent->_pool. + // Because the deconstruction of NodeChannel may take a long time to wait rpc finish. + // but the ObjectPool will hold a spin lock to delete objects. + channel = std::make_shared(_parent, this, replica_node_id); + _node_channels.emplace(replica_node_id, channel); + } else { + channel = it->second; + } + channel->add_tablet(tablet); + if (_parent->_write_single_replica) { + auto slave_location = _parent->_slave_location->find_tablet(tablet.tablet_id); + if (slave_location != nullptr) { + channel->add_slave_tablet_nodes(tablet.tablet_id, slave_location->node_ids); + } + } + channels.push_back(channel); + _tablets_by_channel[replica_node_id].insert(tablet.tablet_id); + } + _channels_by_tablet.emplace(tablet.tablet_id, std::move(channels)); + } + for (auto& it : _node_channels) { + RETURN_IF_ERROR(it.second->init(state)); + } + if (_where_clause != nullptr) { + RETURN_IF_ERROR(_where_clause->prepare(state, *_parent->_output_row_desc)); + RETURN_IF_ERROR(_where_clause->open(state)); + } + return Status::OK(); +} + +void IndexChannel::mark_as_failed(const VNodeChannel* node_channel, const std::string& err, + int64_t tablet_id) { + DCHECK(node_channel != nullptr); + LOG(INFO) << "mark node_id:" << node_channel->channel_info() << " tablet_id: " << tablet_id + << " as failed, err: " << err; + auto node_id = node_channel->node_id(); + const auto& it = _tablets_by_channel.find(node_id); + if (it == _tablets_by_channel.end()) { + return; + } + + { + std::lock_guard l(_fail_lock); + if (tablet_id == -1) { + for (const auto the_tablet_id : it->second) { + _failed_channels[the_tablet_id].insert(node_id); + _failed_channels_msgs.emplace(the_tablet_id, + err + ", host: " + node_channel->host()); + if (_failed_channels[the_tablet_id].size() >= ((_parent->_num_replicas + 1) / 2)) { + _intolerable_failure_status = + Status::InternalError(_failed_channels_msgs[the_tablet_id]); + } + } + } else { + _failed_channels[tablet_id].insert(node_id); + _failed_channels_msgs.emplace(tablet_id, err + ", host: " + node_channel->host()); + if (_failed_channels[tablet_id].size() >= ((_parent->_num_replicas + 1) / 2)) { + _intolerable_failure_status = + Status::InternalError(_failed_channels_msgs[tablet_id]); + } + } + } +} + +Status IndexChannel::check_intolerable_failure() { + std::lock_guard l(_fail_lock); + return _intolerable_failure_status; +} + +void IndexChannel::set_error_tablet_in_state(RuntimeState* state) { + std::vector& error_tablet_infos = state->error_tablet_infos(); + + std::lock_guard l(_fail_lock); + for (const auto& it : _failed_channels_msgs) { + TErrorTabletInfo error_info; + error_info.__set_tabletId(it.first); + error_info.__set_msg(it.second); + error_tablet_infos.emplace_back(error_info); + } +} + +void IndexChannel::set_tablets_received_rows( + const std::vector>& tablets_received_rows, int64_t node_id) { + for (const auto& [tablet_id, rows_num] : tablets_received_rows) { + _tablets_received_rows[tablet_id].emplace_back(node_id, rows_num); + } +} + +void IndexChannel::set_tablets_filtered_rows( + const std::vector>& tablets_filtered_rows, int64_t node_id) { + for (const auto& [tablet_id, rows_num] : tablets_filtered_rows) { + _tablets_filtered_rows[tablet_id].emplace_back(node_id, rows_num); + } +} + +Status IndexChannel::check_tablet_received_rows_consistency() { + for (auto& tablet : _tablets_received_rows) { + for (size_t i = 0; i < tablet.second.size(); i++) { + VLOG_NOTICE << "check_tablet_received_rows_consistency, load_id: " << _parent->_load_id + << ", txn_id: " << std::to_string(_parent->_txn_id) + << ", tablet_id: " << tablet.first + << ", node_id: " << tablet.second[i].first + << ", rows_num: " << tablet.second[i].second; + if (i == 0) { + continue; + } + if (tablet.second[i].second != tablet.second[0].second) { + return Status::InternalError( + "rows num written by multi replicas doest't match, load_id={}, txn_id={}, " + "tablt_id={}, node_id={}, rows_num={}, node_id={}, rows_num={}", + print_id(_parent->_load_id), _parent->_txn_id, tablet.first, + tablet.second[i].first, tablet.second[i].second, tablet.second[0].first, + tablet.second[0].second); + } + } + } + return Status::OK(); +} + +Status IndexChannel::check_tablet_filtered_rows_consistency() { + for (auto& tablet : _tablets_filtered_rows) { + for (size_t i = 0; i < tablet.second.size(); i++) { + VLOG_NOTICE << "check_tablet_filtered_rows_consistency, load_id: " << _parent->_load_id + << ", txn_id: " << std::to_string(_parent->_txn_id) + << ", tablet_id: " << tablet.first + << ", node_id: " << tablet.second[i].first + << ", rows_num: " << tablet.second[i].second; + if (i == 0) { + continue; + } + if (tablet.second[i].second != tablet.second[0].second) { + return Status::InternalError( + "rows num filtered by multi replicas doest't match, load_id={}, txn_id={}, " + "tablt_id={}, node_id={}, rows_num={}, node_id={}, rows_num={}", + print_id(_parent->_load_id), _parent->_txn_id, tablet.first, + tablet.second[i].first, tablet.second[i].second, tablet.second[0].first, + tablet.second[0].second); + } + } + } + return Status::OK(); +} + +VNodeChannel::VNodeChannel(VTabletWriter* parent, IndexChannel* index_channel, int64_t node_id, + bool is_incremental) + : _parent(parent), + _index_channel(index_channel), + _node_id(node_id), + _is_incremental(is_incremental) { + _node_channel_tracker = std::make_shared(fmt::format( + "NodeChannel:indexID={}:threadId={}", std::to_string(_index_channel->_index_id), + thread_context()->get_thread_id())); +} + +VNodeChannel::~VNodeChannel() { + for (auto& closure : _open_closures) { + if (closure != nullptr) { + if (closure->unref()) { + delete closure; + } + closure = nullptr; + } + } + if (_add_block_closure != nullptr) { + delete _add_block_closure; + _add_block_closure = nullptr; + } + static_cast(_cur_add_block_request.release_id()); +} + +void VNodeChannel::clear_all_blocks() { + std::lock_guard lg(_pending_batches_lock); + std::queue empty; + std::swap(_pending_blocks, empty); + _cur_mutable_block.reset(); +} + +// if "_cancelled" is set to true, +// no need to set _cancel_msg because the error will be +// returned directly via "TabletSink::prepare()" method. +Status VNodeChannel::init(RuntimeState* state) { + SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); + _tuple_desc = _parent->_output_tuple_desc; + _state = state; + // get corresponding BE node. + auto node = _parent->_nodes_info->find_node(_node_id); + if (node == nullptr) { + _cancelled = true; + return Status::InternalError("unknown node id, id={}", _node_id); + } + _node_info = *node; + + _load_info = "load_id=" + print_id(_parent->_load_id) + + ", txn_id=" + std::to_string(_parent->_txn_id); + + _row_desc.reset(new RowDescriptor(_tuple_desc, false)); + _batch_size = state->batch_size(); + + _stub = state->exec_env()->brpc_internal_client_cache()->get_client(_node_info.host, + _node_info.brpc_port); + if (_stub == nullptr) { + _cancelled = true; + return Status::InternalError("Get rpc stub failed, host={}, port={}, info={}", + _node_info.host, _node_info.brpc_port, channel_info()); + } + + _rpc_timeout_ms = state->execution_timeout() * 1000; + _timeout_watch.start(); + + // Initialize _cur_add_block_request + if (!_cur_add_block_request.has_id()) { + _cur_add_block_request.set_allocated_id(&_parent->_load_id); + } + _cur_add_block_request.set_index_id(_index_channel->_index_id); + _cur_add_block_request.set_sender_id(_parent->_sender_id); + _cur_add_block_request.set_backend_id(_node_id); + _cur_add_block_request.set_eos(false); + + _name = fmt::format("VNodeChannel[{}-{}]", _index_channel->_index_id, _node_id); + // The node channel will send _batch_size rows of data each rpc. When the + // number of tablets is large, the number of data rows received by each + // tablet is small, TabletsChannel need to traverse each tablet for import. + // so the import performance is poor. Therefore, we set _batch_size to + // a relatively large value to improve the import performance. + _batch_size = std::max(_batch_size, 8192); + + return Status::OK(); +} + +void VNodeChannel::_open_internal(bool is_incremental) { + SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); + PTabletWriterOpenRequest request; + request.set_allocated_id(&_parent->_load_id); + request.set_index_id(_index_channel->_index_id); + request.set_txn_id(_parent->_txn_id); + request.set_allocated_schema(_parent->_schema->to_protobuf()); + std::set deduper; + for (auto& tablet : _all_tablets) { + if (deduper.contains(tablet.tablet_id)) { + continue; + } + auto ptablet = request.add_tablets(); + ptablet->set_partition_id(tablet.partition_id); + ptablet->set_tablet_id(tablet.tablet_id); + deduper.insert(tablet.tablet_id); + } + request.set_num_senders(_parent->_num_senders); + request.set_need_gen_rollup(false); // Useless but it is a required field in pb + request.set_load_mem_limit(_parent->_load_mem_limit); + request.set_load_channel_timeout_s(_parent->_load_channel_timeout_s); + request.set_is_high_priority(_parent->_is_high_priority); + request.set_sender_ip(BackendOptions::get_localhost()); + request.set_is_vectorized(true); + request.set_backend_id(_node_id); + request.set_enable_profile(_state->enable_profile()); + request.set_is_incremental(is_incremental); + + auto* open_closure = new RefCountClosure {}; + open_closure->ref(); + + open_closure->ref(); // This ref is for RPC's reference + open_closure->cntl.set_timeout_ms(config::tablet_writer_open_rpc_timeout_sec * 1000); + if (config::tablet_writer_ignore_eovercrowded) { + open_closure->cntl.ignore_eovercrowded(); + } + // the real transmission here. the corresponding BE's load mgr will open load channel for it. + _stub->tablet_writer_open(&open_closure->cntl, &request, &open_closure->result, open_closure); + _open_closures.push_back(open_closure); + + static_cast(request.release_id()); + static_cast(request.release_schema()); +} + +void VNodeChannel::open() { + _open_internal(false); +} + +void VNodeChannel::incremental_open() { + _open_internal(true); +} + +Status VNodeChannel::open_wait() { + Status status; + for (auto& open_closure : _open_closures) { + // because of incremental open, we will wait multi times. so skip the closures which have been checked and set to nullptr in previous rounds + if (open_closure == nullptr) { + continue; + } + + open_closure->join(); + SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); + if (open_closure->cntl.Failed()) { + if (!ExecEnv::GetInstance()->brpc_internal_client_cache()->available( + _stub, _node_info.host, _node_info.brpc_port)) { + ExecEnv::GetInstance()->brpc_internal_client_cache()->erase( + open_closure->cntl.remote_side()); + } + + _cancelled = true; + auto error_code = open_closure->cntl.ErrorCode(); + auto error_text = open_closure->cntl.ErrorText(); + if (open_closure->unref()) { + delete open_closure; + } + open_closure = nullptr; + return Status::InternalError( + "failed to open tablet writer, error={}, error_text={}, info={}", + berror(error_code), error_text, channel_info()); + } + status = Status::create(open_closure->result.status()); + if (open_closure->unref()) { + delete open_closure; + } + open_closure = nullptr; + + if (!status.ok()) { + _cancelled = true; + return status; + } + } + + // add block closure + _add_block_closure = ReusableClosure::create(); + _add_block_closure->addFailedHandler( + [this](bool is_last_rpc) { _add_block_failed_callback(is_last_rpc); }); + + _add_block_closure->addSuccessHandler( + [this](const PTabletWriterAddBlockResult& result, bool is_last_rpc) { + _add_block_success_callback(result, is_last_rpc); + }); + return status; +} + +Status VNodeChannel::add_block(vectorized::Block* block, const Payload* payload, bool is_append) { + SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); + if (payload->second.empty()) { + return Status::OK(); + } + // If add_block() when _eos_is_produced==true, there must be sth wrong, we can only mark this channel as failed. + auto st = none_of({_cancelled, _eos_is_produced}); + if (!st.ok()) { + if (_cancelled) { + std::lock_guard l(_cancel_msg_lock); + return Status::InternalError("add row failed. {}", _cancel_msg); + } else { + return std::move(st.prepend("already stopped, can't add row. cancelled/eos: ")); + } + } + + // We use OlapTableSink mem_tracker which has the same ancestor of _plan node, + // so in the ideal case, mem limit is a matter for _plan node. + // But there is still some unfinished things, we do mem limit here temporarily. + // _cancelled may be set by rpc callback, and it's possible that _cancelled might be set in any of the steps below. + // It's fine to do a fake add_block() and return OK, because we will check _cancelled in next add_block() or mark_close(). + while (!_cancelled && _pending_batches_num > 0 && + _pending_batches_bytes > _max_pending_batches_bytes) { + SCOPED_RAW_TIMER(&_stat.mem_exceeded_block_ns); + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + } + + if (UNLIKELY(!_cur_mutable_block)) { + _cur_mutable_block = vectorized::MutableBlock::create_unique(block->clone_empty()); + } + + std::unique_ptr temp_payload = nullptr; + if (_index_channel != nullptr && _index_channel->get_where_clause() != nullptr) { + SCOPED_RAW_TIMER(&_stat.where_clause_ns); + temp_payload.reset(new Payload( + std::unique_ptr(new vectorized::IColumn::Selector()), + std::vector())); + int result_index = -1; + size_t column_number = block->columns(); + RETURN_IF_ERROR(_index_channel->get_where_clause()->execute(block, &result_index)); + + auto& row_ids = *payload->first; + auto& tablets_ids = payload->second; + + auto filter_column = block->get_by_position(result_index).column; + + if (auto* nullable_column = + vectorized::check_and_get_column(*filter_column)) { + for (size_t i = 0; i < payload->second.size(); i++) { + if (nullable_column->get_bool_inline(row_ids[i])) { + temp_payload->first->emplace_back(row_ids[i]); + temp_payload->second.emplace_back(tablets_ids[i]); + } + } + payload = temp_payload.get(); + } else if (auto* const_column = vectorized::check_and_get_column( + *filter_column)) { + bool ret = const_column->get_bool(0); + if (!ret) { + return Status::OK(); + } + } else { + auto& filter = assert_cast(*filter_column).get_data(); + for (size_t i = 0; i < payload->second.size(); i++) { + if (filter[row_ids[i]] != 0) { + temp_payload->first->emplace_back(row_ids[i]); + temp_payload->second.emplace_back(tablets_ids[i]); + } + } + payload = temp_payload.get(); + } + + for (size_t i = block->columns() - 1; i >= column_number; i--) { + block->erase(i); + } + } + + SCOPED_RAW_TIMER(&_stat.append_node_channel_ns); + if (is_append) { + // Do not split the data of the block by tablets but append it to a single delta writer. + // This is a faster way to send block than append_to_block_by_selector + // TODO: we could write to local delta writer if single_replica_load is true + VLOG_DEBUG << "send whole block by append block"; + std::vector tablets(block->rows(), payload->second[0]); + vectorized::MutableColumns& columns = _cur_mutable_block->mutable_columns(); + columns.clear(); + columns.reserve(block->columns()); + // Hold the reference of block columns to avoid copying + for (auto column : block->get_columns()) { + columns.push_back(std::move(*column).mutate()); + } + *_cur_add_block_request.mutable_tablet_ids() = {tablets.begin(), tablets.end()}; + _cur_add_block_request.set_is_single_tablet_block(true); + } else { + block->append_to_block_by_selector(_cur_mutable_block.get(), *(payload->first)); + for (auto tablet_id : payload->second) { + _cur_add_block_request.add_tablet_ids(tablet_id); + } + } + + if (is_append || _cur_mutable_block->rows() >= _batch_size || + _cur_mutable_block->bytes() > config::doris_scanner_row_bytes) { + { + SCOPED_ATOMIC_TIMER(&_queue_push_lock_ns); + std::lock_guard l(_pending_batches_lock); + // To simplify the add_row logic, postpone adding block into req until the time of sending req + _pending_batches_bytes += _cur_mutable_block->allocated_bytes(); + _cur_add_block_request.set_eos( + false); // for multi-add, only when marking close we set it eos. + _pending_blocks.emplace(std::move(_cur_mutable_block), _cur_add_block_request); + _pending_batches_num++; + VLOG_DEBUG << "VTabletWriter:" << _parent << " VNodeChannel:" << this + << " pending_batches_bytes:" << _pending_batches_bytes + << " jobid:" << std::to_string(_state->load_job_id()) + << " loadinfo:" << _load_info; + } + _cur_mutable_block = vectorized::MutableBlock::create_unique(block->clone_empty()); + _cur_add_block_request.clear_tablet_ids(); + } + + return Status::OK(); +} + +int VNodeChannel::try_send_and_fetch_status(RuntimeState* state, + std::unique_ptr& thread_pool_token) { + auto st = none_of({_cancelled, _send_finished}); + if (!st.ok()) { + return 0; + } + + if (!_add_block_closure->try_set_in_flight()) { + // There is packet in flight, skip. + return _send_finished ? 0 : 1; + } + + // We are sure that try_send_batch is not running + if (_pending_batches_num > 0) { + auto s = thread_pool_token->submit_func( + std::bind(&VNodeChannel::try_send_pending_block, this, state)); + if (!s.ok()) { + _cancel_with_msg("submit send_batch task to send_batch_thread_pool failed"); + // clear in flight + _add_block_closure->clear_in_flight(); + } + // in_flight is cleared in closure::Run + } else { + // clear in flight + _add_block_closure->clear_in_flight(); + } + return _send_finished ? 0 : 1; +} + +void VNodeChannel::_cancel_with_msg(const std::string& msg) { + LOG(WARNING) << "cancel node channel " << channel_info() << ", error message: " << msg; + { + std::lock_guard l(_cancel_msg_lock); + if (_cancel_msg == "") { + _cancel_msg = msg; + } + } + _cancelled = true; +} + +Status VNodeChannel::none_of(std::initializer_list vars) { + bool none = std::none_of(vars.begin(), vars.end(), [](bool var) { return var; }); + Status st = Status::OK(); + if (!none) { + std::string vars_str; + std::for_each(vars.begin(), vars.end(), + [&vars_str](bool var) -> void { vars_str += (var ? "1/" : "0/"); }); + if (!vars_str.empty()) { + vars_str.pop_back(); // 0/1/0/ -> 0/1/0 + } + st = Status::Uninitialized(vars_str); + } + + return st; +} + +void VNodeChannel::try_send_pending_block(RuntimeState* state) { + SCOPED_ATTACH_TASK(state); + SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker); + SCOPED_ATOMIC_TIMER(&_actual_consume_ns); + AddBlockReq send_block; + { + std::lock_guard l(_pending_batches_lock); + DCHECK(!_pending_blocks.empty()); + send_block = std::move(_pending_blocks.front()); + _pending_blocks.pop(); + _pending_batches_num--; + _pending_batches_bytes -= send_block.first->allocated_bytes(); + } + + auto mutable_block = std::move(send_block.first); + auto request = std::move(send_block.second); // doesn't need to be saved in heap + + // tablet_ids has already set when add row + request.set_packet_seq(_next_packet_seq); + auto block = mutable_block->to_block(); + CHECK(block.rows() == request.tablet_ids_size()) + << "block rows: " << block.rows() << ", tablet_ids_size: " << request.tablet_ids_size(); + if (block.rows() > 0) { + SCOPED_ATOMIC_TIMER(&_serialize_batch_ns); + size_t uncompressed_bytes = 0, compressed_bytes = 0; + Status st = block.serialize(state->be_exec_version(), request.mutable_block(), + &uncompressed_bytes, &compressed_bytes, + state->fragement_transmission_compression_type(), + _parent->_transfer_large_data_by_brpc); + if (!st.ok()) { + cancel(fmt::format("{}, err: {}", channel_info(), st.to_string())); + _add_block_closure->clear_in_flight(); + return; + } + if (compressed_bytes >= double(config::brpc_max_body_size) * 0.95f) { + LOG(WARNING) << "send block too large, this rpc may failed. send size: " + << compressed_bytes << ", threshold: " << config::brpc_max_body_size + << ", " << channel_info(); + } + } + + int remain_ms = _rpc_timeout_ms - _timeout_watch.elapsed_time() / NANOS_PER_MILLIS; + if (UNLIKELY(remain_ms < config::min_load_rpc_timeout_ms)) { + if (remain_ms <= 0 && !request.eos()) { + cancel(fmt::format("{}, err: timeout", channel_info())); + _add_block_closure->clear_in_flight(); + return; + } else { + remain_ms = config::min_load_rpc_timeout_ms; + } + } + + _add_block_closure->reset(); + _add_block_closure->cntl.set_timeout_ms(remain_ms); + if (config::tablet_writer_ignore_eovercrowded) { + _add_block_closure->cntl.ignore_eovercrowded(); + } + + if (request.eos()) { + for (auto pid : _parent->_tablet_finder->partition_ids()) { + request.add_partition_ids(pid); + } + + request.set_write_single_replica(false); + if (_parent->_write_single_replica) { + request.set_write_single_replica(true); + for (std::unordered_map>::iterator iter = + _slave_tablet_nodes.begin(); + iter != _slave_tablet_nodes.end(); iter++) { + PSlaveTabletNodes slave_tablet_nodes; + for (auto node_id : iter->second) { + auto node = _parent->_nodes_info->find_node(node_id); + if (node == nullptr) { + return; + } + PNodeInfo* pnode = slave_tablet_nodes.add_slave_nodes(); + pnode->set_id(node->id); + pnode->set_option(node->option); + pnode->set_host(node->host); + pnode->set_async_internal_port(node->brpc_port); + } + request.mutable_slave_tablet_nodes()->insert({iter->first, slave_tablet_nodes}); + } + } + + // eos request must be the last request + _add_block_closure->end_mark(); + _send_finished = true; + CHECK(_pending_batches_num == 0) << _pending_batches_num; + } + + if (_parent->_transfer_large_data_by_brpc && request.has_block() && + request.block().has_column_values() && request.ByteSizeLong() > MIN_HTTP_BRPC_SIZE) { + Status st = request_embed_attachment_contain_block< + PTabletWriterAddBlockRequest, ReusableClosure>( + &request, _add_block_closure); + if (!st.ok()) { + cancel(fmt::format("{}, err: {}", channel_info(), st.to_string())); + _add_block_closure->clear_in_flight(); + return; + } + + //format an ipv6 address + std::string brpc_url = get_brpc_http_url(_node_info.host, _node_info.brpc_port); + std::shared_ptr _brpc_http_stub = + _state->exec_env()->brpc_internal_client_cache()->get_new_client_no_cache(brpc_url, + "http"); + _add_block_closure->cntl.http_request().uri() = + brpc_url + "/PInternalServiceImpl/tablet_writer_add_block_by_http"; + _add_block_closure->cntl.http_request().set_method(brpc::HTTP_METHOD_POST); + _add_block_closure->cntl.http_request().set_content_type("application/json"); + + { + SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker()); + _brpc_http_stub->tablet_writer_add_block_by_http(&_add_block_closure->cntl, nullptr, + &_add_block_closure->result, + _add_block_closure); + } + } else { + _add_block_closure->cntl.http_request().Clear(); + { + SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker()); + _stub->tablet_writer_add_block(&_add_block_closure->cntl, &request, + &_add_block_closure->result, _add_block_closure); + } + } + + _next_packet_seq++; +} + +void VNodeChannel::_add_block_success_callback(const PTabletWriterAddBlockResult& result, + bool is_last_rpc) { + std::lock_guard l(this->_closed_lock); + if (this->_is_closed) { + // if the node channel is closed, no need to call the following logic, + // and notice that _index_channel may already be destroyed. + return; + } + SCOPED_ATTACH_TASK(_state); + Status status(Status::create(result.status())); + if (status.ok()) { + // if has error tablet, handle them first + for (auto& error : result.tablet_errors()) { + _index_channel->mark_as_failed(this, "tablet error: " + error.msg(), error.tablet_id()); + } + + Status st = _index_channel->check_intolerable_failure(); + if (!st.ok()) { + _cancel_with_msg(st.to_string()); + } else if (is_last_rpc) { + for (auto& tablet : result.tablet_vec()) { + TTabletCommitInfo commit_info; + commit_info.tabletId = tablet.tablet_id(); + commit_info.backendId = _node_id; + _tablet_commit_infos.emplace_back(std::move(commit_info)); + if (tablet.has_received_rows()) { + _tablets_received_rows.emplace_back(tablet.tablet_id(), tablet.received_rows()); + } + if (tablet.has_num_rows_filtered()) { + _tablets_filtered_rows.emplace_back(tablet.tablet_id(), + tablet.num_rows_filtered()); + } + VLOG_CRITICAL << "master replica commit info: tabletId=" << tablet.tablet_id() + << ", backendId=" << _node_id + << ", master node id: " << this->node_id() + << ", host: " << this->host() << ", txn_id=" << _parent->_txn_id; + } + if (_parent->_write_single_replica) { + for (auto& tablet_slave_node_ids : result.success_slave_tablet_node_ids()) { + for (auto slave_node_id : tablet_slave_node_ids.second.slave_node_ids()) { + TTabletCommitInfo commit_info; + commit_info.tabletId = tablet_slave_node_ids.first; + commit_info.backendId = slave_node_id; + _tablet_commit_infos.emplace_back(std::move(commit_info)); + VLOG_CRITICAL + << "slave replica commit info: tabletId=" + << tablet_slave_node_ids.first << ", backendId=" << slave_node_id + << ", master node id: " << this->node_id() + << ", host: " << this->host() << ", txn_id=" << _parent->_txn_id; + } + } + } + _add_batches_finished = true; + } + } else { + _cancel_with_msg(fmt::format("{}, add batch req success but status isn't ok, err: {}", + channel_info(), status.to_string())); + } + + if (result.has_execution_time_us()) { + _add_batch_counter.add_batch_execution_time_us += result.execution_time_us(); + _add_batch_counter.add_batch_wait_execution_time_us += result.wait_execution_time_us(); + _add_batch_counter.add_batch_num++; + } + if (result.has_load_channel_profile()) { + TRuntimeProfileTree tprofile; + const uint8_t* buf = (const uint8_t*)result.load_channel_profile().data(); + uint32_t len = result.load_channel_profile().size(); + auto st = deserialize_thrift_msg(buf, &len, false, &tprofile); + if (st.ok()) { + _state->load_channel_profile()->update(tprofile); + } else { + LOG(WARNING) << "load channel TRuntimeProfileTree deserialize failed, errmsg=" << st; + } + } +} + +void VNodeChannel::_add_block_failed_callback(bool is_last_rpc) { + std::lock_guard l(this->_closed_lock); + if (this->_is_closed) { + // if the node channel is closed, no need to call `mark_as_failed`, + // and notice that _index_channel may already be destroyed. + return; + } + SCOPED_ATTACH_TASK(_state); + // If rpc failed, mark all tablets on this node channel as failed + _index_channel->mark_as_failed( + this, + fmt::format("rpc failed, error coed:{}, error text:{}", + _add_block_closure->cntl.ErrorCode(), _add_block_closure->cntl.ErrorText()), + -1); + Status st = _index_channel->check_intolerable_failure(); + if (!st.ok()) { + _cancel_with_msg(fmt::format("{}, err: {}", channel_info(), st.to_string())); + } else if (is_last_rpc) { + // if this is last rpc, will must set _add_batches_finished. otherwise, node channel's close_wait + // will be blocked. + _add_batches_finished = true; + } +} + +void VNodeChannel::cancel(const std::string& cancel_msg) { + if (_is_closed) { + // skip the channels that have been canceled or close_wait. + return; + } + SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); + // set _is_closed to true finally + Defer set_closed {[&]() { + std::lock_guard l(_closed_lock); + _is_closed = true; + }}; + // we don't need to wait last rpc finished, cause closure's release/reset will join. + // But do we need brpc::StartCancel(call_id)? + _cancel_with_msg(cancel_msg); + + PTabletWriterCancelRequest request; + request.set_allocated_id(&_parent->_load_id); + request.set_index_id(_index_channel->_index_id); + request.set_sender_id(_parent->_sender_id); + + auto closure = new RefCountClosure(); + + closure->ref(); + int remain_ms = _rpc_timeout_ms - _timeout_watch.elapsed_time() / NANOS_PER_MILLIS; + if (UNLIKELY(remain_ms < config::min_load_rpc_timeout_ms)) { + remain_ms = config::min_load_rpc_timeout_ms; + } + closure->cntl.set_timeout_ms(remain_ms); + if (config::tablet_writer_ignore_eovercrowded) { + closure->cntl.ignore_eovercrowded(); + } + _stub->tablet_writer_cancel(&closure->cntl, &request, &closure->result, closure); + static_cast(request.release_id()); +} + +bool VNodeChannel::is_send_data_rpc_done() const { + return _add_batches_finished || _cancelled; +} + +Status VNodeChannel::close_wait(RuntimeState* state) { + SCOPED_CONSUME_MEM_TRACKER(_node_channel_tracker.get()); + // set _is_closed to true finally + Defer set_closed {[&]() { + std::lock_guard l(_closed_lock); + _is_closed = true; + }}; + + auto st = none_of({_cancelled, !_eos_is_produced}); + if (!st.ok()) { + if (_cancelled) { + std::lock_guard l(_cancel_msg_lock); + return Status::InternalError("wait close failed. {}", _cancel_msg); + } else { + return std::move( + st.prepend("already stopped, skip waiting for close. cancelled/!eos: ")); + } + } + + // waiting for finished, it may take a long time, so we couldn't set a timeout + // In pipeline, is_close_done() is false at this time, will not bock. + while (!_add_batches_finished && !_cancelled && !state->is_cancelled()) { + bthread_usleep(1000); + } + _close_time_ms = UnixMillis() - _close_time_ms; + + if (_add_batches_finished) { + _close_check(); + state->tablet_commit_infos().insert(state->tablet_commit_infos().end(), + std::make_move_iterator(_tablet_commit_infos.begin()), + std::make_move_iterator(_tablet_commit_infos.end())); + + _index_channel->set_error_tablet_in_state(state); + _index_channel->set_tablets_received_rows(_tablets_received_rows, _node_id); + _index_channel->set_tablets_filtered_rows(_tablets_filtered_rows, _node_id); + return Status::OK(); + } + + return Status::InternalError(get_cancel_msg()); +} + +void VNodeChannel::_close_check() { + std::lock_guard lg(_pending_batches_lock); + CHECK(_pending_blocks.empty()) << name(); + CHECK(_cur_mutable_block == nullptr) << name(); +} + +void VNodeChannel::mark_close() { + auto st = none_of({_cancelled, _eos_is_produced}); + if (!st.ok()) { + return; + } + + _cur_add_block_request.set_eos(true); + { + std::lock_guard l(_pending_batches_lock); + if (!_cur_mutable_block) { + // add a dummy block + _cur_mutable_block = vectorized::MutableBlock::create_unique(); + } + // when prepare to close, add block to queue so that try_send_pending_block thread will send it. + _pending_blocks.emplace(std::move(_cur_mutable_block), _cur_add_block_request); + _pending_batches_num++; + DCHECK(_pending_blocks.back().second.eos()); + _close_time_ms = UnixMillis(); + LOG(INFO) << channel_info() + << " mark closed, left pending batch size: " << _pending_blocks.size(); + } + + _eos_is_produced = true; +} + +VTabletWriter::VTabletWriter(const TDataSink& t_sink, const VExprContextSPtrs& output_exprs) + : AsyncResultWriter(output_exprs), _t_sink(t_sink) { + _transfer_large_data_by_brpc = config::transfer_large_data_by_brpc; +} + +void VTabletWriter::init_properties(doris::ObjectPool* pool, bool group_commit) { + _pool = pool; + _group_commit = group_commit; +} + +void VTabletWriter::_send_batch_process() { + SCOPED_TIMER(_non_blocking_send_timer); + SCOPED_ATTACH_TASK(_state); + SCOPED_CONSUME_MEM_TRACKER(_mem_tracker); + + bool had_effect = false; + while (true) { + // incremental open will temporarily make channels into abnormal state. stop checking when this. + std::unique_lock l(_stop_check_channel); + + int running_channels_num = 0; + for (const auto& index_channel : _channels) { + index_channel->for_each_node_channel([&running_channels_num, + this](const std::shared_ptr& ch) { + running_channels_num += + ch->try_send_and_fetch_status(_state, this->_send_batch_thread_pool_token); + }); + } + + // if there is no channel, maybe auto partition table. so check does there have had running channels ever. + if (running_channels_num == 0 && had_effect) { + LOG(INFO) << "all node channels are stopped(maybe finished/offending/cancelled), " + "sender thread exit. " + << print_id(_load_id); + return; + } else if (running_channels_num != 0) { + had_effect = true; + } + bthread_usleep(config::olap_table_sink_send_interval_ms * 1000); + } +} + +static void* periodic_send_batch(void* writer) { + auto* tablet_writer = (VTabletWriter*)(writer); + tablet_writer->_send_batch_process(); + return nullptr; +} + +Status VTabletWriter::open(doris::RuntimeState* state, doris::RuntimeProfile* profile) { + RETURN_IF_ERROR(_init(state, profile)); + SCOPED_TIMER(profile->total_time_counter()); + SCOPED_TIMER(_open_timer); + SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); + + fmt::memory_buffer buf; + for (const auto& index_channel : _channels) { + fmt::format_to(buf, "index id:{}", index_channel->_index_id); + index_channel->for_each_node_channel( + [](const std::shared_ptr& ch) { ch->open(); }); + } + VLOG_DEBUG << "list of open index id = " << fmt::to_string(buf); + + for (const auto& index_channel : _channels) { + index_channel->for_each_node_channel([&index_channel]( + const std::shared_ptr& ch) { + auto st = ch->open_wait(); + if (!st.ok()) { + // The open() phase is mainly to generate DeltaWriter instances on the nodes corresponding to each node channel. + // This phase will not fail due to a single tablet. + // Therefore, if the open() phase fails, all tablets corresponding to the node need to be marked as failed. + index_channel->mark_as_failed( + ch.get(), + fmt::format("{}, open failed, err: {}", ch->channel_info(), st.to_string()), + -1); + } + }); + + RETURN_IF_ERROR(index_channel->check_intolerable_failure()); + } + int32_t send_batch_parallelism = + MIN(_send_batch_parallelism, config::max_send_batch_parallelism_per_job); + _send_batch_thread_pool_token = state->exec_env()->send_batch_thread_pool()->new_token( + ThreadPool::ExecutionMode::CONCURRENT, send_batch_parallelism); + + // start to send batch continually + if (bthread_start_background(&_sender_thread, nullptr, periodic_send_batch, (void*)this) != 0) { + return Status::Error("bthread_start_backgroud failed"); + } + return Status::OK(); +} + +Status VTabletWriter::_init(RuntimeState* state, RuntimeProfile* profile) { + DCHECK(_t_sink.__isset.olap_table_sink); + auto& table_sink = _t_sink.olap_table_sink; + _load_id.set_hi(table_sink.load_id.hi); + _load_id.set_lo(table_sink.load_id.lo); + _txn_id = table_sink.txn_id; + _num_replicas = table_sink.num_replicas; + _tuple_desc_id = table_sink.tuple_id; + _schema.reset(new OlapTableSchemaParam()); + RETURN_IF_ERROR(_schema->init(table_sink.schema)); + _location = _pool->add(new OlapTableLocationParam(table_sink.location)); + _nodes_info = _pool->add(new DorisNodesInfo(table_sink.nodes_info)); + if (table_sink.__isset.write_single_replica && table_sink.write_single_replica) { + _write_single_replica = true; + _slave_location = _pool->add(new OlapTableLocationParam(table_sink.slave_location)); + if (!config::enable_single_replica_load) { + return Status::InternalError("single replica load is disabled on BE."); + } + } + + if (table_sink.__isset.load_channel_timeout_s) { + _load_channel_timeout_s = table_sink.load_channel_timeout_s; + } else { + _load_channel_timeout_s = config::streaming_load_rpc_max_alive_time_sec; + } + if (table_sink.__isset.send_batch_parallelism && table_sink.send_batch_parallelism > 1) { + _send_batch_parallelism = table_sink.send_batch_parallelism; + } + // if distributed column list is empty, we can ensure that tablet is with random distribution info + // and if load_to_single_tablet is set and set to true, we should find only one tablet in one partition + // for the whole olap table sink + auto find_tablet_mode = OlapTabletFinder::FindTabletMode::FIND_TABLET_EVERY_ROW; + if (table_sink.partition.distributed_columns.empty()) { + if (table_sink.__isset.load_to_single_tablet && table_sink.load_to_single_tablet) { + find_tablet_mode = OlapTabletFinder::FindTabletMode::FIND_TABLET_EVERY_SINK; + } else { + find_tablet_mode = OlapTabletFinder::FindTabletMode::FIND_TABLET_EVERY_BATCH; + } + } + _vpartition = _pool->add(new doris::VOlapTablePartitionParam(_schema, table_sink.partition)); + _tablet_finder = std::make_unique(_vpartition, find_tablet_mode); + RETURN_IF_ERROR(_vpartition->init()); + + _state = state; + _profile = profile; + + _sender_id = state->per_fragment_instance_idx(); + _num_senders = state->num_per_fragment_instances(); + _is_high_priority = + (state->execution_timeout() <= config::load_task_high_priority_threshold_second); + + // profile must add to state's object pool + _mem_tracker = + std::make_shared("OlapTableSink:" + std::to_string(state->load_job_id())); + SCOPED_TIMER(profile->total_time_counter()); + SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); + + // get table's tuple descriptor + _output_tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_desc_id); + if (_output_tuple_desc == nullptr) { + LOG(WARNING) << "unknown destination tuple descriptor, id=" << _tuple_desc_id; + return Status::InternalError("unknown destination tuple descriptor"); + } + + _block_convertor = std::make_unique(_output_tuple_desc); + _block_convertor->init_autoinc_info(_schema->db_id(), _schema->table_id(), + _state->batch_size()); + _output_row_desc = _pool->add(new RowDescriptor(_output_tuple_desc, false)); + + // add all counter + _input_rows_counter = ADD_COUNTER(profile, "RowsRead", TUnit::UNIT); + _output_rows_counter = ADD_COUNTER(profile, "RowsReturned", TUnit::UNIT); + _filtered_rows_counter = ADD_COUNTER(profile, "RowsFiltered", TUnit::UNIT); + _send_data_timer = ADD_TIMER(profile, "SendDataTime"); + _wait_mem_limit_timer = ADD_CHILD_TIMER(profile, "WaitMemLimitTime", "SendDataTime"); + _row_distribution_timer = ADD_CHILD_TIMER(profile, "RowDistributionTime", "SendDataTime"); + _filter_timer = ADD_CHILD_TIMER(profile, "FilterTime", "SendDataTime"); + _where_clause_timer = ADD_CHILD_TIMER(profile, "WhereClauseTime", "SendDataTime"); + _append_node_channel_timer = ADD_CHILD_TIMER(profile, "AppendNodeChannelTime", "SendDataTime"); + _add_partition_request_timer = + ADD_CHILD_TIMER(profile, "AddPartitionRequestTime", "SendDataTime"); + _validate_data_timer = ADD_TIMER(profile, "ValidateDataTime"); + _open_timer = ADD_TIMER(profile, "OpenTime"); + _close_timer = ADD_TIMER(profile, "CloseWaitTime"); + _non_blocking_send_timer = ADD_TIMER(profile, "NonBlockingSendTime"); + _non_blocking_send_work_timer = + ADD_CHILD_TIMER(profile, "NonBlockingSendWorkTime", "NonBlockingSendTime"); + _serialize_batch_timer = + ADD_CHILD_TIMER(profile, "SerializeBatchTime", "NonBlockingSendWorkTime"); + _total_add_batch_exec_timer = ADD_TIMER(profile, "TotalAddBatchExecTime"); + _max_add_batch_exec_timer = ADD_TIMER(profile, "MaxAddBatchExecTime"); + _total_wait_exec_timer = ADD_TIMER(profile, "TotalWaitExecTime"); + _max_wait_exec_timer = ADD_TIMER(profile, "MaxWaitExecTime"); + _add_batch_number = ADD_COUNTER(profile, "NumberBatchAdded", TUnit::UNIT); + _num_node_channels = ADD_COUNTER(profile, "NumberNodeChannels", TUnit::UNIT); + _load_mem_limit = state->get_load_mem_limit(); + +#ifdef DEBUG + // check: tablet ids should be unique + { + std::unordered_set tablet_ids; + const auto& partitions = _vpartition->get_partitions(); + for (int i = 0; i < _schema->indexes().size(); ++i) { + for (const auto& partition : partitions) { + for (const auto& tablet : partition->indexes[i].tablets) { + CHECK(tablet_ids.count(tablet) == 0) << "found duplicate tablet id: " << tablet; + tablet_ids.insert(tablet); + } + } + } + } +#endif + + // open all channels + const auto& partitions = _vpartition->get_partitions(); + for (int i = 0; i < _schema->indexes().size(); ++i) { + // collect all tablets belong to this rollup + std::vector tablets; + auto index = _schema->indexes()[i]; + for (const auto& part : partitions) { + for (const auto& tablet : part->indexes[i].tablets) { + TTabletWithPartition tablet_with_partition; + tablet_with_partition.partition_id = part->id; + tablet_with_partition.tablet_id = tablet; + tablets.emplace_back(std::move(tablet_with_partition)); + } + } + if (tablets.empty() && !_vpartition->is_auto_partition()) { + LOG(WARNING) << "load job:" << state->load_job_id() << " index: " << index->index_id + << " would open 0 tablet"; + } + _channels.emplace_back(new IndexChannel(this, index->index_id, index->where_clause)); + _index_id_to_channel[index->index_id] = _channels.back(); + RETURN_IF_ERROR(_channels.back()->init(state, tablets)); + } + + // prepare for auto partition functions + if (_vpartition->is_auto_partition()) { + auto [part_ctx, part_func] = _get_partition_function(); + RETURN_IF_ERROR(part_func->prepare(_state, *_output_row_desc, part_ctx.get())); + } + + _prepare = true; + return Status::OK(); +} + +Status VTabletWriter::_automatic_create_partition() { + SCOPED_TIMER(_add_partition_request_timer); + TCreatePartitionRequest request; + TCreatePartitionResult result; + request.__set_txn_id(_txn_id); + request.__set_db_id(_vpartition->db_id()); + request.__set_table_id(_vpartition->table_id()); + request.__set_partitionValues(_partitions_need_create); + + VLOG(1) << "automatic partition rpc begin request " << request; + TNetworkAddress master_addr = ExecEnv::GetInstance()->master_info()->network_address; + int time_out = _state->execution_timeout() * 1000; + RETURN_IF_ERROR(ThriftRpcHelper::rpc( + master_addr.hostname, master_addr.port, + [&request, &result](FrontendServiceConnection& client) { + client->createPartition(result, request); + }, + time_out)); + + Status status(Status::create(result.status)); + VLOG(1) << "automatic partition rpc end response " << result; + if (result.status.status_code == TStatusCode::OK) { + // add new created partitions + RETURN_IF_ERROR(_vpartition->add_partitions(result.partitions)); + + // add new tablet locations. it will use by address. so add to pool + auto* new_locations = _pool->add(new std::vector(result.tablets)); + _location->add_locations(*new_locations); + + // update new node info + _nodes_info->add_nodes(result.nodes); + + // incremental open node channel + RETURN_IF_ERROR(_incremental_open_node_channel(result.partitions)); + } + + return status; +} + +Status VTabletWriter::_incremental_open_node_channel( + const std::vector& partitions) { + // do what we did in prepare() for partitions. indexes which don't change when we create new partition is orthogonal to partitions. + std::unique_lock _l(_stop_check_channel); + for (int i = 0; i < _schema->indexes().size(); ++i) { + const OlapTableIndexSchema* index = _schema->indexes()[i]; + std::vector tablets; + for (auto& t_part : partitions) { + VOlapTablePartition* part = nullptr; + RETURN_IF_ERROR(_vpartition->generate_partition_from(t_part, part)); + for (const auto& tablet : part->indexes[i].tablets) { + TTabletWithPartition tablet_with_partition; + tablet_with_partition.partition_id = part->id; + tablet_with_partition.tablet_id = tablet; + tablets.emplace_back(std::move(tablet_with_partition)); + } + DCHECK(!tablets.empty()) << "incremental open got nothing!"; + } + // update and reinit for existing channels. + std::shared_ptr channel = _index_id_to_channel[index->index_id]; + DCHECK(channel != nullptr); + RETURN_IF_ERROR(channel->init(_state, tablets)); // add tablets into it + } + + fmt::memory_buffer buf; + for (auto& channel : _channels) { + // incremental open new partition's tablet on storage side + channel->for_each_node_channel( + [](const std::shared_ptr& ch) { ch->incremental_open(); }); + fmt::format_to(buf, "index id:{}", channel->_index_id); + VLOG_DEBUG << "list of open index id = " << fmt::to_string(buf); + + channel->for_each_node_channel([&channel](const std::shared_ptr& ch) { + auto st = ch->open_wait(); + if (!st.ok()) { + // The open() phase is mainly to generate DeltaWriter instances on the nodes corresponding to each node channel. + // This phase will not fail due to a single tablet. + // Therefore, if the open() phase fails, all tablets corresponding to the node need to be marked as failed. + channel->mark_as_failed( + ch.get(), + fmt::format("{}, open failed, err: {}", ch->channel_info(), st.to_string()), + -1); + } + }); + + RETURN_IF_ERROR(channel->check_intolerable_failure()); + } + + return Status::OK(); +} + +void VTabletWriter::_generate_row_distribution_payload( + ChannelDistributionPayload& channel_to_payload, const VOlapTablePartition* partition, + uint32_t tablet_index, int row_idx, size_t row_cnt) { + // Generate channel payload for sinking data to differenct node channel + for (int j = 0; j < partition->indexes.size(); ++j) { + auto tid = partition->indexes[j].tablets[tablet_index]; + auto it = _channels[j]->_channels_by_tablet.find(tid); + DCHECK(it != _channels[j]->_channels_by_tablet.end()) + << "unknown tablet, tablet_id=" << tablet_index; + for (const auto& channel : it->second) { + if (channel_to_payload[j].count(channel.get()) < 1) { + channel_to_payload[j].insert( + {channel.get(), Payload {std::unique_ptr( + new vectorized::IColumn::Selector()), + std::vector()}}); + } + channel_to_payload[j][channel.get()].first->push_back(row_idx); + channel_to_payload[j][channel.get()].second.push_back(tid); + } + _number_output_rows += row_cnt; + } +} + +Status VTabletWriter::_single_partition_generate(RuntimeState* state, vectorized::Block* block, + ChannelDistributionPayload& channel_to_payload, + size_t num_rows, bool has_filtered_rows) { + const VOlapTablePartition* partition = nullptr; + uint32_t tablet_index = 0; + bool stop_processing = false; + for (int32_t i = 0; i < num_rows; ++i) { + if (UNLIKELY(has_filtered_rows) && _block_convertor->filter_bitmap().Get(i)) { + continue; + } + bool is_continue = false; + RETURN_IF_ERROR(_tablet_finder->find_tablet(state, block, i, &partition, tablet_index, + stop_processing, is_continue)); + if (is_continue) { + continue; + } + break; + } + if (partition == nullptr) { + return Status::OK(); + } + for (int j = 0; j < partition->indexes.size(); ++j) { + auto tid = partition->indexes[j].tablets[tablet_index]; + auto it = _channels[j]->_channels_by_tablet.find(tid); + DCHECK(it != _channels[j]->_channels_by_tablet.end()) + << "unknown tablet, tablet_id=" << tablet_index; + int64_t row_cnt = 0; + for (const auto& channel : it->second) { + if (channel_to_payload[j].count(channel.get()) < 1) { + channel_to_payload[j].insert( + {channel.get(), Payload {std::unique_ptr( + new vectorized::IColumn::Selector()), + std::vector()}}); + } + auto& selector = channel_to_payload[j][channel.get()].first; + auto& tablet_ids = channel_to_payload[j][channel.get()].second; + for (int32_t i = 0; i < num_rows; ++i) { + if (UNLIKELY(has_filtered_rows) && _block_convertor->filter_bitmap().Get(i)) { + continue; + } + selector->push_back(i); + } + tablet_ids.resize(selector->size(), tid); + row_cnt = selector->size(); + } + _number_output_rows += row_cnt; + } + return Status::OK(); +} + +std::pair +VTabletWriter::_get_partition_function() { + return {_vpartition->get_part_func_ctx(), _vpartition->get_partition_function()}; +} + +void VTabletWriter::_save_missing_values(vectorized::ColumnPtr col, + vectorized::DataTypePtr value_type, + std::vector filter) { + _partitions_need_create.clear(); + std::set deduper; + // de-duplication + for (auto row : filter) { + deduper.emplace(value_type->to_string(*col, row)); + } + for (auto& value : deduper) { + TStringLiteral node; + node.value = value; + _partitions_need_create.emplace_back(std::vector {node}); // only 1 partition column now + } +} + +Status VTabletWriter::_cancel_channel_and_check_intolerable_failure( + Status status, const std::string& err_msg, const std::shared_ptr ich, + const std::shared_ptr nch) { + LOG(WARNING) << nch->channel_info() << ", close channel failed, err: " << err_msg; + ich->mark_as_failed(nch.get(), err_msg, -1); + // cancel the node channel in best effort + nch->cancel(err_msg); + + // check if index has intolerable failure + Status index_st = ich->check_intolerable_failure(); + if (!index_st.ok()) { + status = index_st; + } else if (Status st = ich->check_tablet_received_rows_consistency(); !st.ok()) { + status = st; + } else if (Status st = ich->check_tablet_filtered_rows_consistency(); !st.ok()) { + status = st; + } + return status; +} + +void VTabletWriter::_cancel_all_channel(Status status) { + for (const auto& index_channel : _channels) { + index_channel->for_each_node_channel([&status](const std::shared_ptr& ch) { + ch->cancel(status.to_string()); + }); + } + LOG(INFO) << fmt::format( + "close olap table sink. load_id={}, txn_id={}, canceled all node channels due to " + "error: {}", + print_id(_load_id), _txn_id, status); +} + +Status VTabletWriter::try_close(RuntimeState* state, Status exec_status) { + SCOPED_TIMER(_close_timer); + Status status = exec_status; + if (status.ok()) { + // only if status is ok can we call this _profile->total_time_counter(). + // if status is not ok, this sink may not be prepared, so that _profile is null + SCOPED_TIMER(_profile->total_time_counter()); + { + for (const auto& index_channel : _channels) { + if (!status.ok()) { + break; + } + index_channel->for_each_node_channel( + [this, &index_channel, &status](const std::shared_ptr& ch) { + if (!status.ok() || ch->is_closed()) { + return; + } + // only first try close, all node channels will mark_close() + ch->mark_close(); + if (ch->is_cancelled()) { + status = this->_cancel_channel_and_check_intolerable_failure( + status, ch->get_cancel_msg(), index_channel, ch); + } + }); + } // end for index channels + } + } + + if (!status.ok()) { + _cancel_all_channel(status); + _close_status = status; + _try_close = true; + } + + return Status::OK(); +} + +bool VTabletWriter::is_close_done() { + // Only after try_close, need to wait rpc end. + if (!_try_close) { + return true; + } + bool close_done = true; + for (const auto& index_channel : _channels) { + index_channel->for_each_node_channel( + [&close_done](const std::shared_ptr& ch) { + close_done &= ch->is_send_data_rpc_done(); + }); + } + return close_done; +} + +Status VTabletWriter::close(Status exec_status) { + if (!_prepare) { + DCHECK(!exec_status.ok()); + _cancel_all_channel(exec_status); + _close_status = exec_status; + return _close_status; + } + + SCOPED_TIMER(_close_timer); + SCOPED_TIMER(_profile->total_time_counter()); + + try_close(_state, exec_status); + + // If _close_status is not ok, all nodes have been canceled in try_close. + if (_close_status.ok()) { + auto status = Status::OK(); + // BE id -> add_batch method counter + std::unordered_map node_add_batch_counter_map; + int64_t serialize_batch_ns = 0, queue_push_lock_ns = 0, actual_consume_ns = 0, + total_add_batch_exec_time_ns = 0, max_add_batch_exec_time_ns = 0, + total_wait_exec_time_ns = 0, max_wait_exec_time_ns = 0, total_add_batch_num = 0, + num_node_channels = 0; + VNodeChannelStat channel_stat; + + for (const auto& index_channel : _channels) { + if (!status.ok()) { + break; + } + int64_t add_batch_exec_time = 0; + int64_t wait_exec_time = 0; + index_channel->for_each_node_channel( + [this, &index_channel, &status, &node_add_batch_counter_map, + &serialize_batch_ns, &channel_stat, &queue_push_lock_ns, &actual_consume_ns, + &total_add_batch_exec_time_ns, &add_batch_exec_time, &total_wait_exec_time_ns, + &wait_exec_time, + &total_add_batch_num](const std::shared_ptr& ch) { + if (!status.ok() || ch->is_closed()) { + return; + } + // in pipeline, all node channels are done or canceled, will not block. + // no pipeline, close may block waiting. + auto s = ch->close_wait(_state); + if (!s.ok()) { + status = this->_cancel_channel_and_check_intolerable_failure( + status, s.to_string(), index_channel, ch); + } + ch->time_report(&node_add_batch_counter_map, &serialize_batch_ns, + &channel_stat, &queue_push_lock_ns, &actual_consume_ns, + &total_add_batch_exec_time_ns, &add_batch_exec_time, + &total_wait_exec_time_ns, &wait_exec_time, + &total_add_batch_num); + }); + + // Due to the non-determinism of compaction, the rowsets of each replica may be different from each other on different + // BE nodes. The number of rows filtered in SegmentWriter depends on the historical rowsets located in the correspoding + // BE node. So we check the number of rows filtered on each succeccful BE to ensure the consistency of the current load + if (status.ok() && !_write_single_replica && _schema->is_strict_mode() && + _schema->is_partial_update()) { + if (Status st = index_channel->check_tablet_filtered_rows_consistency(); !st.ok()) { + status = st; + } else { + _state->set_num_rows_filtered_in_strict_mode_partial_update( + index_channel->num_rows_filtered()); + } + } + + num_node_channels += index_channel->num_node_channels(); + if (add_batch_exec_time > max_add_batch_exec_time_ns) { + max_add_batch_exec_time_ns = add_batch_exec_time; + } + if (wait_exec_time > max_wait_exec_time_ns) { + max_wait_exec_time_ns = wait_exec_time; + } + } // end for index channels + + if (status.ok()) { + // TODO need to be improved + LOG(INFO) << "total mem_exceeded_block_ns=" << channel_stat.mem_exceeded_block_ns + << ", total queue_push_lock_ns=" << queue_push_lock_ns + << ", total actual_consume_ns=" << actual_consume_ns + << ", load id=" << print_id(_load_id); + + COUNTER_SET(_input_rows_counter, _number_input_rows); + COUNTER_SET(_output_rows_counter, _number_output_rows); + COUNTER_SET(_filtered_rows_counter, + _block_convertor->num_filtered_rows() + + _tablet_finder->num_filtered_rows() + + _state->num_rows_filtered_in_strict_mode_partial_update()); + COUNTER_SET(_send_data_timer, _send_data_ns); + COUNTER_SET(_row_distribution_timer, (int64_t)_row_distribution_watch.elapsed_time()); + COUNTER_SET(_filter_timer, _filter_ns); + COUNTER_SET(_append_node_channel_timer, channel_stat.append_node_channel_ns); + COUNTER_SET(_where_clause_timer, channel_stat.where_clause_ns); + COUNTER_SET(_wait_mem_limit_timer, channel_stat.mem_exceeded_block_ns); + COUNTER_SET(_validate_data_timer, _block_convertor->validate_data_ns()); + COUNTER_SET(_serialize_batch_timer, serialize_batch_ns); + COUNTER_SET(_non_blocking_send_work_timer, actual_consume_ns); + COUNTER_SET(_total_add_batch_exec_timer, total_add_batch_exec_time_ns); + COUNTER_SET(_max_add_batch_exec_timer, max_add_batch_exec_time_ns); + COUNTER_SET(_total_wait_exec_timer, total_wait_exec_time_ns); + COUNTER_SET(_max_wait_exec_timer, max_wait_exec_time_ns); + COUNTER_SET(_add_batch_number, total_add_batch_num); + COUNTER_SET(_num_node_channels, num_node_channels); + // _number_input_rows don't contain num_rows_load_filtered and num_rows_load_unselected in scan node + int64_t num_rows_load_total = _number_input_rows + _state->num_rows_load_filtered() + + _state->num_rows_load_unselected(); + _state->set_num_rows_load_total(num_rows_load_total); + _state->update_num_rows_load_filtered( + _block_convertor->num_filtered_rows() + _tablet_finder->num_filtered_rows() + + _state->num_rows_filtered_in_strict_mode_partial_update()); + _state->update_num_rows_load_unselected( + _tablet_finder->num_immutable_partition_filtered_rows()); + + // print log of add batch time of all node, for tracing load performance easily + std::stringstream ss; + ss << "finished to close olap table sink. load_id=" << print_id(_load_id) + << ", txn_id=" << _txn_id + << ", node add batch time(ms)/wait execution time(ms)/close time(ms)/num: "; + for (auto const& pair : node_add_batch_counter_map) { + ss << "{" << pair.first << ":(" << (pair.second.add_batch_execution_time_us / 1000) + << ")(" << (pair.second.add_batch_wait_execution_time_us / 1000) << ")(" + << pair.second.close_wait_time_ms << ")(" << pair.second.add_batch_num << ")} "; + } + LOG(INFO) << ss.str(); + } else { + _cancel_all_channel(status); + } + _close_status = status; + } + + // Sender join() must put after node channels mark_close/cancel. + // But there is no specific sequence required between sender join() & close_wait(). + if (_sender_thread) { + bthread_join(_sender_thread, nullptr); + // We have to wait all task in _send_batch_thread_pool_token finished, + // because it is difficult to handle concurrent problem if we just + // shutdown it. + _send_batch_thread_pool_token->wait(); + } + + // We clear NodeChannels' batches here, cuz NodeChannels' batches destruction will use + // OlapTableSink::_mem_tracker and its parents. + // But their destructions are after OlapTableSink's. + for (const auto& index_channel : _channels) { + index_channel->for_each_node_channel( + [](const std::shared_ptr& ch) { ch->clear_all_blocks(); }); + } + + return _close_status; +} + +Status VTabletWriter::append_block(doris::vectorized::Block& input_block) { + SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); + Status status = Status::OK(); + + if (_state->query_options().dry_run_query) { + return status; + } + + auto rows = input_block.rows(); + auto bytes = input_block.bytes(); + if (UNLIKELY(rows == 0)) { + return status; + } + SCOPED_TIMER(_profile->total_time_counter()); + _number_input_rows += rows; + // update incrementally so that FE can get the progress. + // the real 'num_rows_load_total' will be set when sink being closed. + _state->update_num_rows_load_total(rows); + _state->update_num_bytes_load_total(bytes); + DorisMetrics::instance()->load_rows->increment(rows); + DorisMetrics::instance()->load_bytes->increment(bytes); + + std::shared_ptr block; + bool has_filtered_rows = false; + int64_t filtered_rows = + _block_convertor->num_filtered_rows() + _tablet_finder->num_filtered_rows(); + RETURN_IF_ERROR(_block_convertor->validate_and_convert_block( + _state, &input_block, block, _vec_output_expr_ctxs, rows, has_filtered_rows)); + + SCOPED_RAW_TIMER(&_send_data_ns); + // This is just for passing compilation. + bool stop_processing = false; + std::vector> channel_to_payload; + channel_to_payload.resize(_channels.size()); + _tablet_finder->clear_for_new_batch(); + _row_distribution_watch.start(); + auto num_rows = block->rows(); + size_t partition_num = _vpartition->get_partitions().size(); + if (!_vpartition->is_auto_partition() && partition_num == 1 && + _tablet_finder->is_find_tablet_every_sink()) { + RETURN_IF_ERROR(_single_partition_generate(_state, block.get(), channel_to_payload, + num_rows, has_filtered_rows)); + } else { + // if there's projection of partition calc, we need to calc it first. + auto [part_ctx, part_func] = _get_partition_function(); + int result_idx; + if (_vpartition->is_projection_partition()) { + // calc the start value of missing partition ranges. + part_func->execute(part_ctx.get(), block.get(), &result_idx); + VLOG_DEBUG << "Partition-calculated block:" << block->dump_data(); + // change the column to compare to transformed. + _vpartition->set_transformed_slots({(uint16_t)result_idx}); + } + + if (_vpartition->is_auto_partition()) { + std::vector partition_keys = _vpartition->get_partition_keys(); + //TODO: use loop to create missing_vals for multi column. + CHECK(partition_keys.size() == 1) + << "now support only 1 partition column for auto partitions."; + auto partition_col = block->get_by_position(partition_keys[0]); + + std::vector missing_map; // indice of missing values in partition_col + missing_map.reserve(partition_col.column->size()); + + // try to find tablet and save missing value + for (int i = 0; i < num_rows; ++i) { + if (UNLIKELY(has_filtered_rows) && _block_convertor->filter_bitmap().Get(i)) { + continue; + } + const VOlapTablePartition* partition = nullptr; + bool is_continue = false; + uint32_t tablet_index = 0; + bool missing_this = false; + RETURN_IF_ERROR(_tablet_finder->find_tablet(_state, block.get(), i, &partition, + tablet_index, stop_processing, + is_continue, &missing_this)); + if (missing_this) { + missing_map.push_back(i); + } else { + _generate_row_distribution_payload(channel_to_payload, partition, tablet_index, + i, 1); + } + } + missing_map.shrink_to_fit(); + + // for missing partition keys, calc the missing partition and save in _partitions_need_create + auto type = partition_col.type; + if (missing_map.size() > 0) { + auto return_type = part_func->data_type(); + + // expose the data column + vectorized::ColumnPtr range_left_col = block->get_by_position(result_idx).column; + if (auto* nullable = + check_and_get_column(*range_left_col)) { + range_left_col = nullable->get_nested_column_ptr(); + return_type = + assert_cast(return_type.get()) + ->get_nested_type(); + } + // calc the end value and save them. + _save_missing_values(range_left_col, return_type, missing_map); + // then call FE to create it. then FragmentExecutor will redo the load. + RETURN_IF_ERROR(_automatic_create_partition()); + // now we need to rollback the metrics + _number_input_rows -= rows; + _state->update_num_rows_load_total(-rows); + _state->update_num_bytes_load_total(-bytes); + DorisMetrics::instance()->load_rows->increment(-rows); + DorisMetrics::instance()->load_bytes->increment(-bytes); + // In the next round, we will _generate_row_distribution_payload again to get right payload of new tablet + LOG(INFO) << "Auto created partition. Send block again."; + return Status::NeedSendAgain(""); + } // creating done + } else { // not auto partition + for (int i = 0; i < num_rows; ++i) { + if (UNLIKELY(has_filtered_rows) && _block_convertor->filter_bitmap().Get(i)) { + continue; + } + const VOlapTablePartition* partition = nullptr; + bool is_continue = false; + uint32_t tablet_index = 0; + RETURN_IF_ERROR(_tablet_finder->find_tablet(_state, block.get(), i, &partition, + tablet_index, stop_processing, + is_continue)); + if (is_continue) { + continue; + } + // each row + _generate_row_distribution_payload(channel_to_payload, partition, tablet_index, i, + 1); + } + } + } + _row_distribution_watch.stop(); + // Random distribution and the block belongs to a single tablet, we could optimize to append the whole + // block into node channel. + bool load_block_to_single_tablet = + !_vpartition->is_auto_partition() && _tablet_finder->is_single_tablet(); + if (load_block_to_single_tablet) { + SCOPED_RAW_TIMER(&_filter_ns); + // Filter block + if (has_filtered_rows) { + auto filter = vectorized::ColumnUInt8::create(block->rows(), 0); + vectorized::UInt8* filter_data = + static_cast(filter.get())->get_data().data(); + vectorized::IColumn::Filter& filter_col = + static_cast(filter.get())->get_data(); + for (size_t i = 0; i < filter_col.size(); ++i) { + filter_data[i] = !_block_convertor->filter_bitmap().Get(i); + } + RETURN_IF_CATCH_EXCEPTION(vectorized::Block::filter_block_internal( + block.get(), filter_col, block->columns())); + } + } + + if (_group_commit) { + _group_commit_block(&input_block, rows, + _block_convertor->num_filtered_rows() + + _tablet_finder->num_filtered_rows() - filtered_rows); + } + // TODO: Before load, we need to projection unuseful column + // auto slots = _schema->tuple_desc()->slots(); + // for (auto desc : slots) { + // desc->col_pos(); + // } + // Add block to node channel + for (size_t i = 0; i < _channels.size(); i++) { + for (const auto& entry : channel_to_payload[i]) { + // if this node channel is already failed, this add_row will be skipped + auto st = entry.first->add_block( + block.get(), &entry.second, // entry.second is a [row -> tablet] mapping + // if it is load single tablet, then append this whole block + load_block_to_single_tablet); + if (!st.ok()) { + _channels[i]->mark_as_failed(entry.first, st.to_string()); + } + } + } + + // check intolerable failure + for (const auto& index_channel : _channels) { + RETURN_IF_ERROR(index_channel->check_intolerable_failure()); + } + return Status::OK(); +} + +void VTabletWriter::_group_commit_block(Block* input_block, int64_t rows, int64_t filter_rows) { + auto* future_block = assert_cast(input_block); + std::unique_lock l(*(future_block->lock)); + future_block->set_result(Status::OK(), rows, rows - filter_rows); + future_block->cv->notify_all(); +} + +} // namespace vectorized +} // namespace doris diff --git a/be/src/vec/sink/writer/vtablet_writer.h b/be/src/vec/sink/writer/vtablet_writer.h new file mode 100644 index 00000000000000..1f0f62bb945186 --- /dev/null +++ b/be/src/vec/sink/writer/vtablet_writer.h @@ -0,0 +1,677 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +// IWYU pragma: no_include +#include // IWYU pragma: keep +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common/config.h" +#include "common/status.h" +#include "exec/data_sink.h" +#include "exec/tablet_info.h" +#include "gutil/ref_counted.h" +#include "runtime/decimalv2_value.h" +#include "runtime/exec_env.h" +#include "runtime/memory/mem_tracker.h" +#include "runtime/thread_context.h" +#include "runtime/types.h" +#include "util/countdown_latch.h" +#include "util/runtime_profile.h" +#include "util/spinlock.h" +#include "util/stopwatch.hpp" +#include "vec/columns/column.h" +#include "vec/common/allocator.h" +#include "vec/core/block.h" +#include "vec/data_types/data_type.h" +#include "vec/exprs/vexpr_fwd.h" +#include "vec/runtime/vfile_writer_wrapper.h" +#include "vec/sink/vtablet_block_convertor.h" +#include "vec/sink/vtablet_finder.h" +#include "vec/sink/writer/async_result_writer.h" + +namespace doris { +class ObjectPool; +class RowDescriptor; +class RuntimeState; +class TDataSink; +class TExpr; +class Thread; +class ThreadPoolToken; +class TupleDescriptor; +template +class RefCountClosure; + +namespace vectorized { + +// The counter of add_batch rpc of a single node +struct AddBatchCounter { + // total execution time of a add_batch rpc + int64_t add_batch_execution_time_us = 0; + // lock waiting time in a add_batch rpc + int64_t add_batch_wait_execution_time_us = 0; + // number of add_batch call + int64_t add_batch_num = 0; + // time passed between marked close and finish close + int64_t close_wait_time_ms = 0; + + AddBatchCounter& operator+=(const AddBatchCounter& rhs) { + add_batch_execution_time_us += rhs.add_batch_execution_time_us; + add_batch_wait_execution_time_us += rhs.add_batch_wait_execution_time_us; + add_batch_num += rhs.add_batch_num; + close_wait_time_ms += rhs.close_wait_time_ms; + return *this; + } + friend AddBatchCounter operator+(const AddBatchCounter& lhs, const AddBatchCounter& rhs) { + AddBatchCounter sum = lhs; + sum += rhs; + return sum; + } +}; + +// It's very error-prone to guarantee the handler capture vars' & this closure's destruct sequence. +// So using create() to get the closure pointer is recommended. We can delete the closure ptr before the capture vars destruction. +// Delete this point is safe, don't worry about RPC callback will run after ReusableClosure deleted. +// "Ping-Pong" between sender and receiver, `try_set_in_flight` when send, `clear_in_flight` after rpc failure or callback, +// then next send will start, and it will wait for the rpc callback to complete when it is destroyed. +template +class ReusableClosure final : public google::protobuf::Closure { +public: + ReusableClosure() : cid(INVALID_BTHREAD_ID) {} + ~ReusableClosure() override { + // shouldn't delete when Run() is calling or going to be called, wait for current Run() done. + join(); + SCOPED_TRACK_MEMORY_TO_UNKNOWN(); + cntl.Reset(); + } + + static ReusableClosure* create() { return new ReusableClosure(); } + + void addFailedHandler(const std::function& fn) { failed_handler = fn; } + void addSuccessHandler(const std::function& fn) { success_handler = fn; } + + void join() { + // We rely on in_flight to assure one rpc is running, + // while cid is not reliable due to memory order. + // in_flight is written before getting callid, + // so we can not use memory fence to synchronize. + while (_packet_in_flight) { + // cid here is complicated + if (cid != INVALID_BTHREAD_ID) { + // actually cid may be the last rpc call id. + brpc::Join(cid); + } + if (_packet_in_flight) { + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + } + } + } + + // plz follow this order: reset() -> set_in_flight() -> send brpc batch + void reset() { + SCOPED_TRACK_MEMORY_TO_UNKNOWN(); + cntl.Reset(); + cid = cntl.call_id(); + } + + // if _packet_in_flight == false, set it to true. Return true. + // if _packet_in_flight == true, Return false. + bool try_set_in_flight() { + bool value = false; + return _packet_in_flight.compare_exchange_strong(value, true); + } + + void clear_in_flight() { _packet_in_flight = false; } + + bool is_packet_in_flight() { return _packet_in_flight; } + + void end_mark() { + DCHECK(_is_last_rpc == false); + _is_last_rpc = true; + } + + void Run() override { + DCHECK(_packet_in_flight); + if (cntl.Failed()) { + LOG(WARNING) << "failed to send brpc batch, error=" << berror(cntl.ErrorCode()) + << ", error_text=" << cntl.ErrorText(); + failed_handler(_is_last_rpc); + } else { + success_handler(result, _is_last_rpc); + } + clear_in_flight(); + } + + brpc::Controller cntl; + T result; + +private: + brpc::CallId cid; + std::atomic _packet_in_flight {false}; + std::atomic _is_last_rpc {false}; + std::function failed_handler; + std::function success_handler; +}; + +class IndexChannel; +class VTabletWriter; + +// pair +using Payload = std::pair, std::vector>; + +class VNodeChannelStat { +public: + VNodeChannelStat& operator+=(const VNodeChannelStat& stat) { + mem_exceeded_block_ns += stat.mem_exceeded_block_ns; + where_clause_ns += stat.where_clause_ns; + append_node_channel_ns += stat.append_node_channel_ns; + return *this; + }; + + int64_t mem_exceeded_block_ns = 0; + int64_t where_clause_ns = 0; + int64_t append_node_channel_ns = 0; +}; + +// every NodeChannel keeps a data transmission channel with one BE. for multiple times open, it has a dozen of requests and corresponding closures. +class VNodeChannel { +public: + VNodeChannel(VTabletWriter* parent, IndexChannel* index_channel, int64_t node_id, + bool is_incremental = false); + + ~VNodeChannel(); + + // called before open, used to add tablet located in this backend. called by IndexChannel::init + void add_tablet(const TTabletWithPartition& tablet) { _all_tablets.emplace_back(tablet); } + std::string debug_tablets() const { + std::stringstream ss; + for (auto& tab : _all_tablets) { + tab.printTo(ss); + ss << '\n'; + } + return ss.str(); + } + + void add_slave_tablet_nodes(int64_t tablet_id, const std::vector& slave_nodes) { + _slave_tablet_nodes[tablet_id] = slave_nodes; + } + + // build a request and build corresponding connect to BE. + void open(); + // for auto partition, we use this to open more tablet. + void incremental_open(); + + Status init(RuntimeState* state); + + // this will block until all request transmission which were opened or incremental opened finished. + Status open_wait(); + + Status add_block(vectorized::Block* block, const Payload* payload, bool is_append = false); + + // @return: unfinished running channels. + // @caller: VOlapTabletSink::_send_batch_process. it's a continual asynchronous process. + int try_send_and_fetch_status(RuntimeState* state, + std::unique_ptr& thread_pool_token); + // when there's pending block found by try_send_and_fetch_status(), we will awake a thread to send it. + void try_send_pending_block(RuntimeState* state); + + void clear_all_blocks(); + + // two ways to stop channel: + // 1. mark_close()->close_wait() PS. close_wait() will block waiting for the last AddBatch rpc response. + // 2. just cancel() + void mark_close(); + + bool is_send_data_rpc_done() const; + + bool is_closed() const { return _is_closed; } + bool is_cancelled() const { return _cancelled; } + std::string get_cancel_msg() { + std::stringstream ss; + ss << "close wait failed coz rpc error"; + { + std::lock_guard l(_cancel_msg_lock); + if (_cancel_msg != "") { + ss << ". " << _cancel_msg; + } + } + return ss.str(); + } + + // two ways to stop channel: + // 1. mark_close()->close_wait() PS. close_wait() will block waiting for the last AddBatch rpc response. + // 2. just cancel() + Status close_wait(RuntimeState* state); + + void cancel(const std::string& cancel_msg); + + void time_report(std::unordered_map* add_batch_counter_map, + int64_t* serialize_batch_ns, VNodeChannelStat* stat, + int64_t* queue_push_lock_ns, int64_t* actual_consume_ns, + int64_t* total_add_batch_exec_time_ns, int64_t* add_batch_exec_time_ns, + int64_t* total_wait_exec_time_ns, int64_t* wait_exec_time_ns, + int64_t* total_add_batch_num) const { + (*add_batch_counter_map)[_node_id] += _add_batch_counter; + (*add_batch_counter_map)[_node_id].close_wait_time_ms = _close_time_ms; + *serialize_batch_ns += _serialize_batch_ns; + *stat += _stat; + *queue_push_lock_ns += _queue_push_lock_ns; + *actual_consume_ns += _actual_consume_ns; + *add_batch_exec_time_ns = (_add_batch_counter.add_batch_execution_time_us * 1000); + *total_add_batch_exec_time_ns += *add_batch_exec_time_ns; + *wait_exec_time_ns = (_add_batch_counter.add_batch_wait_execution_time_us * 1000); + *total_wait_exec_time_ns += *wait_exec_time_ns; + *total_add_batch_num += _add_batch_counter.add_batch_num; + } + + int64_t node_id() const { return _node_id; } + std::string host() const { return _node_info.host; } + std::string name() const { return _name; } + + Status none_of(std::initializer_list vars); + + std::string channel_info() const { + return fmt::format("{}, {}, node={}:{}", _name, _load_info, _node_info.host, + _node_info.brpc_port); + } + + size_t get_pending_bytes() { return _pending_batches_bytes; } + + bool is_incremental() const { return _is_incremental; } + +protected: + // make a real open request for relative BE's load channel. + void _open_internal(bool is_incremental); + + void _close_check(); + void _cancel_with_msg(const std::string& msg); + + void _add_block_success_callback(const PTabletWriterAddBlockResult& result, bool is_last_rpc); + void _add_block_failed_callback(bool is_last_rpc); + + VTabletWriter* _parent = nullptr; + IndexChannel* _index_channel = nullptr; + int64_t _node_id = -1; + std::string _load_info; + std::string _name; + + std::shared_ptr _node_channel_tracker; + + TupleDescriptor* _tuple_desc = nullptr; + NodeInfo _node_info; + + // this should be set in init() using config + int _rpc_timeout_ms = 60000; + int64_t _next_packet_seq = 0; + MonotonicStopWatch _timeout_watch; + + // the timestamp when this node channel be marked closed and finished closed + uint64_t _close_time_ms = 0; + + // user cancel or get some errors + std::atomic _cancelled {false}; + doris::SpinLock _cancel_msg_lock; + std::string _cancel_msg; + + // send finished means the consumer thread which send the rpc can exit + std::atomic _send_finished {false}; + + // add batches finished means the last rpc has be response, used to check whether this channel can be closed + std::atomic _add_batches_finished {false}; // reuse for vectorized + + bool _eos_is_produced {false}; // only for restricting producer behaviors + + std::unique_ptr _row_desc; + int _batch_size = 0; + + // limit _pending_batches size + std::atomic _pending_batches_bytes {0}; + size_t _max_pending_batches_bytes {(size_t)config::nodechannel_pending_queue_max_bytes}; + std::mutex _pending_batches_lock; // reuse for vectorized + std::atomic _pending_batches_num {0}; // reuse for vectorized + + std::shared_ptr _stub = nullptr; + // because we have incremantal open, we should keep one relative closure for one request. it's similarly for adding block. + std::vector*> _open_closures; + + std::vector _all_tablets; + // map from tablet_id to node_id where slave replicas locate in + std::unordered_map> _slave_tablet_nodes; + std::vector _tablet_commit_infos; + + AddBatchCounter _add_batch_counter; + std::atomic _serialize_batch_ns {0}; + std::atomic _queue_push_lock_ns {0}; + std::atomic _actual_consume_ns {0}; + + VNodeChannelStat _stat; + // lock to protect _is_closed. + // The methods in the IndexChannel are called back in the RpcClosure in the NodeChannel. + // However, this rpc callback may occur after the whole task is finished (e.g. due to network latency), + // and by that time the IndexChannel may have been destructured, so we should not call the + // IndexChannel methods anymore, otherwise the BE will crash. + // Therefore, we use the _is_closed and _closed_lock to ensure that the RPC callback + // function will not call the IndexChannel method after the NodeChannel is closed. + // The IndexChannel is definitely accessible until the NodeChannel is closed. + std::mutex _closed_lock; + bool _is_closed = false; + + RuntimeState* _state; + // rows number received per tablet, tablet_id -> rows_num + std::vector> _tablets_received_rows; + // rows number filtered per tablet, tablet_id -> filtered_rows_num + std::vector> _tablets_filtered_rows; + + // build a _cur_mutable_block and push into _pending_blocks. when not building, this block is empty. + std::unique_ptr _cur_mutable_block; + PTabletWriterAddBlockRequest _cur_add_block_request; + + using AddBlockReq = + std::pair, PTabletWriterAddBlockRequest>; + std::queue _pending_blocks; + ReusableClosure* _add_block_closure = nullptr; + + bool _is_incremental; +}; + +// an IndexChannel is related to specific table and its rollup and mv +class IndexChannel { +public: + IndexChannel(VTabletWriter* parent, int64_t index_id, + const vectorized::VExprContextSPtr& where_clause) + : _parent(parent), _index_id(index_id), _where_clause(where_clause) { + _index_channel_tracker = + std::make_unique("IndexChannel:indexID=" + std::to_string(_index_id)); + } + ~IndexChannel() = default; + + // allow to init multi times, for incremental open more tablets for one index(table) + Status init(RuntimeState* state, const std::vector& tablets); + + void for_each_node_channel( + const std::function&)>& func) { + for (auto& it : _node_channels) { + func(it.second); + } + } + + void mark_as_failed(const VNodeChannel* node_channel, const std::string& err, + int64_t tablet_id = -1); + Status check_intolerable_failure(); + + // set error tablet info in runtime state, so that it can be returned to FE. + void set_error_tablet_in_state(RuntimeState* state); + + size_t num_node_channels() const { return _node_channels.size(); } + + size_t get_pending_bytes() const { + size_t mem_consumption = 0; + for (auto& kv : _node_channels) { + mem_consumption += kv.second->get_pending_bytes(); + } + return mem_consumption; + } + + void set_tablets_received_rows( + const std::vector>& tablets_received_rows, int64_t node_id); + + void set_tablets_filtered_rows( + const std::vector>& tablets_filtered_rows, int64_t node_id); + + int64_t num_rows_filtered() { + DCHECK(!_tablets_filtered_rows.empty()); + // the Unique table has no roll up or materilized view + // we just add up filtered rows from all partitions + return std::accumulate(_tablets_filtered_rows.cbegin(), _tablets_filtered_rows.cend(), 0, + [](int64_t sum, const auto& a) { return sum + a.second[0].second; }); + } + + // check whether the rows num written by different replicas is consistent + Status check_tablet_received_rows_consistency(); + + // check whether the rows num filtered by different replicas is consistent + Status check_tablet_filtered_rows_consistency(); + + vectorized::VExprContextSPtr get_where_clause() { return _where_clause; } + +private: + friend class VNodeChannel; + friend class VTabletWriter; + + VTabletWriter* _parent; + int64_t _index_id; + vectorized::VExprContextSPtr _where_clause; + + // from backend channel to tablet_id + // ATTN: must be placed before `_node_channels` and `_channels_by_tablet`. + // Because the destruct order of objects is opposite to the creation order. + // So NodeChannel will be destructured first. + // And the destructor function of NodeChannel waits for all RPCs to finish. + // This ensures that it is safe to use `_tablets_by_channel` in the callback function for the end of the RPC. + std::unordered_map> _tablets_by_channel; + // BeId -> channel + std::unordered_map> _node_channels; + // from tablet_id to backend channel + std::unordered_map>> _channels_by_tablet; + + // lock to protect _failed_channels and _failed_channels_msgs + mutable doris::SpinLock _fail_lock; + // key is tablet_id, value is a set of failed node id + std::unordered_map> _failed_channels; + // key is tablet_id, value is error message + std::unordered_map _failed_channels_msgs; + Status _intolerable_failure_status = Status::OK(); + + std::unique_ptr _index_channel_tracker; + // rows num received by DeltaWriter per tablet, tablet_id -> + // used to verify whether the rows num received by different replicas is consistent + std::map>> _tablets_received_rows; + + // rows num filtered by DeltaWriter per tablet, tablet_id -> + // used to verify whether the rows num filtered by different replicas is consistent + std::map>> _tablets_filtered_rows; +}; +} // namespace vectorized +} // namespace doris + +namespace doris::vectorized { +// +// write result to file +class VTabletWriter final : public AsyncResultWriter { +public: + VTabletWriter(const TDataSink& t_sink, const VExprContextSPtrs& output_exprs); + + void init_properties(ObjectPool* pool, bool group_commit); + + Status append_block(Block& block) override; + + Status close(Status) override; + + Status open(RuntimeState* state, RuntimeProfile* profile) override; + + Status try_close(RuntimeState* state, Status exec_status); + + // the consumer func of sending pending batches in every NodeChannel. + // use polling & NodeChannel::try_send_and_fetch_status() to achieve nonblocking sending. + // only focus on pending batches and channel status, the internal errors of NodeChannels will be handled by the producer + void _send_batch_process(); + + bool is_close_done(); + +private: + friend class VNodeChannel; + friend class IndexChannel; + + using ChannelDistributionPayload = std::vector>; + + Status _init(RuntimeState* state, RuntimeProfile* profile); + // payload for each row + void _generate_row_distribution_payload(ChannelDistributionPayload& payload, + const VOlapTablePartition* partition, + uint32_t tablet_index, int row_idx, size_t row_cnt); + Status _single_partition_generate(RuntimeState* state, vectorized::Block* block, + ChannelDistributionPayload& channel_to_payload, + size_t num_rows, bool has_filtered_rows); + + Status _cancel_channel_and_check_intolerable_failure(Status status, const std::string& err_msg, + const std::shared_ptr ich, + const std::shared_ptr nch); + + void _cancel_all_channel(Status status); + + std::pair _get_partition_function(); + + void _save_missing_values(vectorized::ColumnPtr col, vectorized::DataTypePtr value_type, + std::vector filter); + + // create partitions when need for auto-partition table using #_partitions_need_create. + Status _automatic_create_partition(); + + Status _incremental_open_node_channel(const std::vector& partitions); + + void _group_commit_block(vectorized::Block* input_block, int64_t rows, int64_t filter_rows); + + TDataSink _t_sink; + + std::shared_ptr _mem_tracker; + + ObjectPool* _pool; + + bthread_t _sender_thread = 0; + + // unique load id + PUniqueId _load_id; + int64_t _txn_id = -1; + int _num_replicas = -1; + int _tuple_desc_id = -1; + + // this is tuple descriptor of destination OLAP table + TupleDescriptor* _output_tuple_desc = nullptr; + RowDescriptor* _output_row_desc = nullptr; + + // number of senders used to insert into OlapTable, if we only support single node insert, + // all data from select should collectted and then send to OlapTable. + // To support multiple senders, we maintain a channel for each sender. + int _sender_id = -1; + int _num_senders = -1; + bool _is_high_priority = false; + + // TODO(zc): think about cache this data + std::shared_ptr _schema; + OlapTableLocationParam* _location = nullptr; + bool _write_single_replica = false; + OlapTableLocationParam* _slave_location = nullptr; + DorisNodesInfo* _nodes_info = nullptr; + + std::unique_ptr _tablet_finder; + + // index_channel + std::mutex _stop_check_channel; + std::vector> _channels; + std::unordered_map> _index_id_to_channel; + + std::unique_ptr _send_batch_thread_pool_token; + + // support only one partition column now + std::vector> _partitions_need_create; + + std::unique_ptr _block_convertor; + // Stats for this + int64_t _send_data_ns = 0; + int64_t _number_input_rows = 0; + int64_t _number_output_rows = 0; + int64_t _filter_ns = 0; + + MonotonicStopWatch _row_distribution_watch; + + RuntimeProfile::Counter* _input_rows_counter = nullptr; + RuntimeProfile::Counter* _output_rows_counter = nullptr; + RuntimeProfile::Counter* _filtered_rows_counter = nullptr; + RuntimeProfile::Counter* _send_data_timer = nullptr; + RuntimeProfile::Counter* _row_distribution_timer = nullptr; + RuntimeProfile::Counter* _append_node_channel_timer = nullptr; + RuntimeProfile::Counter* _filter_timer = nullptr; + RuntimeProfile::Counter* _where_clause_timer = nullptr; + RuntimeProfile::Counter* _add_partition_request_timer = nullptr; + RuntimeProfile::Counter* _wait_mem_limit_timer = nullptr; + RuntimeProfile::Counter* _validate_data_timer = nullptr; + RuntimeProfile::Counter* _open_timer = nullptr; + RuntimeProfile::Counter* _close_timer = nullptr; + RuntimeProfile::Counter* _non_blocking_send_timer = nullptr; + RuntimeProfile::Counter* _non_blocking_send_work_timer = nullptr; + RuntimeProfile::Counter* _serialize_batch_timer = nullptr; + RuntimeProfile::Counter* _total_add_batch_exec_timer = nullptr; + RuntimeProfile::Counter* _max_add_batch_exec_timer = nullptr; + RuntimeProfile::Counter* _total_wait_exec_timer = nullptr; + RuntimeProfile::Counter* _max_wait_exec_timer = nullptr; + RuntimeProfile::Counter* _add_batch_number = nullptr; + RuntimeProfile::Counter* _num_node_channels = nullptr; + + // load mem limit is for remote load channel + int64_t _load_mem_limit = -1; + + // the timeout of load channels opened by this tablet sink. in second + int64_t _load_channel_timeout_s = 0; + + int32_t _send_batch_parallelism = 1; + // Save the status of try_close() and close() method + Status _close_status; + bool _try_close = false; + bool _prepare = false; + + // User can change this config at runtime, avoid it being modified during query or loading process. + bool _transfer_large_data_by_brpc = false; + + VOlapTablePartitionParam* _vpartition = nullptr; + + RuntimeState* _state = nullptr; // not owned, set when open + RuntimeProfile* _profile = nullptr; // not owned, set when open + bool _group_commit = false; +}; +} // namespace doris::vectorized diff --git a/be/test/testutil/function_utils.cpp b/be/test/testutil/function_utils.cpp index 6a87bb7009edc5..150d81f3399ed7 100644 --- a/be/test/testutil/function_utils.cpp +++ b/be/test/testutil/function_utils.cpp @@ -39,13 +39,17 @@ FunctionUtils::FunctionUtils() { FunctionUtils::FunctionUtils(const doris::TypeDescriptor& return_type, const std::vector& arg_types, - int varargs_buffer_size) { + int varargs_buffer_size, RuntimeState* state = nullptr) { TQueryGlobals globals; globals.__set_now_string("2019-08-06 01:38:57"); globals.__set_timestamp_ms(1565026737805); globals.__set_time_zone("Asia/Shanghai"); - _state = RuntimeState::create_unique(globals).release(); - _fn_ctx = FunctionContext::create_context(_state, return_type, arg_types); + if (state == nullptr) { + _state = RuntimeState::create_unique(globals).release(); + _fn_ctx = FunctionContext::create_context(_state, return_type, arg_types); + } else { + _fn_ctx = FunctionContext::create_context(state, return_type, arg_types); + } } FunctionUtils::~FunctionUtils() { diff --git a/be/test/testutil/function_utils.h b/be/test/testutil/function_utils.h index fbb641eb079c13..f952ae0ed255cb 100644 --- a/be/test/testutil/function_utils.h +++ b/be/test/testutil/function_utils.h @@ -29,7 +29,8 @@ class FunctionUtils { public: FunctionUtils(); FunctionUtils(const doris::TypeDescriptor& return_type, - const std::vector& arg_types, int varargs_buffer_size); + const std::vector& arg_types, int varargs_buffer_size, + RuntimeState*); ~FunctionUtils(); doris::FunctionContext* get_fn_ctx() { return _fn_ctx.get(); } diff --git a/be/test/vec/exec/delta_writer_v2_pool_test.cpp b/be/test/vec/exec/delta_writer_v2_pool_test.cpp index dfc3276ea75c4b..60c03747853e08 100644 --- a/be/test/vec/exec/delta_writer_v2_pool_test.cpp +++ b/be/test/vec/exec/delta_writer_v2_pool_test.cpp @@ -20,9 +20,7 @@ #include "olap/delta_writer_v2.h" -namespace doris { - -namespace stream_load { +namespace doris::vectorized { class DeltaWriterV2PoolTest : public testing::Test { public: @@ -83,5 +81,4 @@ TEST_F(DeltaWriterV2PoolTest, test_map) { EXPECT_EQ(0, pool.size()); } -} // namespace stream_load -} // namespace doris +} // namespace doris::vectorized diff --git a/be/test/vec/exec/vtablet_sink_test.cpp b/be/test/vec/exec/vtablet_sink_test.cpp index 75ed927fcb2acc..122232c74fead8 100644 --- a/be/test/vec/exec/vtablet_sink_test.cpp +++ b/be/test/vec/exec/vtablet_sink_test.cpp @@ -55,7 +55,7 @@ class RpcController; namespace doris { class PFunctionService_Stub; -namespace stream_load { +namespace vectorized { Status k_add_batch_status; @@ -416,7 +416,8 @@ class VOlapTableSinkTest : public testing::Test { std::set output_set; service->_output_set = &output_set; - VOlapTableSink sink(&obj_pool, row_desc, {}, &st); + std::vector exprs; + VOlapTableSink sink(&obj_pool, row_desc, exprs, false); ASSERT_TRUE(st.ok()); // init @@ -556,7 +557,7 @@ TEST_F(VOlapTableSinkTest, convert) { exprs[2].nodes[0].slot_ref.slot_id = 2; exprs[2].nodes[0].slot_ref.tuple_id = 1; - VOlapTableSink sink(&obj_pool, row_desc, exprs, &st); + VOlapTableSink sink(&obj_pool, row_desc, exprs, false); ASSERT_TRUE(st.ok()); // set output tuple_id @@ -683,7 +684,7 @@ TEST_F(VOlapTableSinkTest, add_block_failed) { exprs[2].nodes[0].slot_ref.slot_id = 2; exprs[2].nodes[0].slot_ref.tuple_id = 1; - VOlapTableSink sink(&obj_pool, row_desc, exprs, &st); + VOlapTableSink sink(&obj_pool, row_desc, exprs, false); ASSERT_TRUE(st.ok()); // set output tuple_id @@ -777,7 +778,8 @@ TEST_F(VOlapTableSinkTest, decimal) { std::set output_set; service->_output_set = &output_set; - VOlapTableSink sink(&obj_pool, row_desc, {}, &st); + std::vector exprs; + VOlapTableSink sink(&obj_pool, row_desc, exprs, false); ASSERT_TRUE(st.ok()); // init @@ -833,5 +835,5 @@ TEST_F(VOlapTableSinkTest, decimal) { ASSERT_TRUE(output_set.count("(12, 12.300000000)") > 0); ASSERT_TRUE(output_set.count("(13, 123.120000000)") > 0); } -} // namespace stream_load +} // namespace vectorized } // namespace doris diff --git a/be/test/vec/function/function_test_util.h b/be/test/vec/function/function_test_util.h index fc9fb8a60dbabb..c543612439afd2 100644 --- a/be/test/vec/function/function_test_util.h +++ b/be/test/vec/function/function_test_util.h @@ -29,6 +29,7 @@ #include "gtest/gtest_pred_impl.h" #include "olap/olap_common.h" #include "runtime/define_primitive_type.h" +#include "runtime/exec_env.h" #include "runtime/types.h" #include "testutil/any_type.h" #include "testutil/function_utils.h" @@ -199,9 +200,11 @@ void check_vec_table_function(TableFunction* fn, const InputTypeSet& input_types // Null values are represented by Null() // The type of the constant column is represented as follows: Consted {TypeIndex::String} // A DataSet with a constant column can only have one row of data +// If state != nullptr, should set query options you use for your own. template Status check_function(const std::string& func_name, const InputTypeSet& input_types, - const DataSet& data_set, bool expect_fail = false) { + const DataSet& data_set, bool expect_fail = false, + RuntimeState* state = nullptr) { // 1.0 create data type ut_type::UTDataTypeDescs descs; EXPECT_TRUE(parse_ut_data_type(input_types, descs)); @@ -270,7 +273,7 @@ Status check_function(const std::string& func_name, const InputTypeSet& input_ty fn_ctx_return.type = doris::PrimitiveType::INVALID_TYPE; } - FunctionUtils fn_utils(fn_ctx_return, arg_types, 0); + FunctionUtils fn_utils(fn_ctx_return, arg_types, 0, state); auto* fn_ctx = fn_utils.get_fn_ctx(); fn_ctx->set_constant_cols(constant_cols); func->open(fn_ctx, FunctionContext::FRAGMENT_LOCAL); diff --git a/be/test/vec/function/function_time_test.cpp b/be/test/vec/function/function_time_test.cpp index e6281bc8f444e9..ba4e352f82af9a 100644 --- a/be/test/vec/function/function_time_test.cpp +++ b/be/test/vec/function/function_time_test.cpp @@ -23,12 +23,12 @@ #include "common/status.h" #include "function_test_util.h" -#include "gtest/gtest_pred_impl.h" +#include "runtime/runtime_state.h" #include "testutil/any_type.h" +#include "util/timezone_utils.h" #include "vec/core/types.h" #include "vec/data_types/data_type_date.h" #include "vec/data_types/data_type_date_time.h" -#include "vec/data_types/data_type_nullable.h" #include "vec/data_types/data_type_number.h" #include "vec/data_types/data_type_string.h" #include "vec/data_types/data_type_time.h" @@ -201,6 +201,62 @@ TEST(VTimestampFunctionsTest, timediff_test) { check_function(func_name, input_types, data_set); } +TEST(VTimestampFunctionsTest, convert_tz_test) { + std::string func_name = "convert_tz"; + + ExecEnv* exec_env = ExecEnv::GetInstance(); + exec_env->_global_zone_cache = std::make_unique(); + auto test_state = RuntimeState::create_unique(); + test_state->set_exec_env(exec_env); + TimezoneUtils::clear_timezone_names(); + + InputTypeSet input_types = {TypeIndex::DateTimeV2, TypeIndex::String, TypeIndex::String}; + + { + DataSet data_set = {{{std::string {"2019-08-01 02:18:27"}, std::string {"Asia/SHANGHAI"}, + std::string {"america/Los_angeles"}}, + Null()}}; + check_function(func_name, input_types, data_set, false, + test_state.get()); + } + + { + DataSet data_set = {{{std::string {"2019-08-01 02:18:27"}, std::string {"Asia/Shanghai"}, + std::string {"UTC"}}, + str_to_datetime_v2("2019-07-31 18:18:27", "%Y-%m-%d %H:%i:%s.%f")}, + {{std::string {"2019-08-01 02:18:27"}, std::string {"Asia/Shanghai"}, + std::string {"Utc"}}, + Null()}, + {{std::string {"2019-08-01 02:18:27"}, std::string {"Asia/Shanghai"}, + std::string {"UTC"}}, + str_to_datetime_v2("2019-07-31 18:18:27", "%Y-%m-%d %H:%i:%s.%f")}, + {{std::string {"2019-08-01 02:18:27"}, std::string {"Asia/SHANGHAI"}, + std::string {"america/Los_angeles"}}, + Null()}}; + check_function(func_name, input_types, data_set, false, + test_state.get()); + } + + { + DataSet data_set = {{{std::string {"2019-08-01 02:18:27"}, std::string {"Asia/Shanghai"}, + std::string {"UTC"}}, + str_to_datetime_v2("2019-07-31 18:18:27", "%Y-%m-%d %H:%i:%s.%f")}, + {{std::string {"2019-08-01 02:18:27"}, std::string {"Asia/Shanghai"}, + std::string {"Utc"}}, + str_to_datetime_v2("2019-07-31 18:18:27", "%Y-%m-%d %H:%i:%s.%f")}, + {{std::string {"2019-08-01 02:18:27"}, std::string {"Asia/Shanghai"}, + std::string {"UTC"}}, + str_to_datetime_v2("2019-07-31 18:18:27", "%Y-%m-%d %H:%i:%s.%f")}, + {{std::string {"2019-08-01 02:18:27"}, std::string {"Asia/SHANGHAI"}, + std::string {"america/Los_angeles"}}, + str_to_datetime_v2("2019-07-31 11:18:27", "%Y-%m-%d %H:%i:%s.%f")}}; + TimezoneUtils::load_timezone_names(); + TimezoneUtils::load_timezones_to_cache(*exec_env->_global_zone_cache); + check_function(func_name, input_types, data_set, false, + test_state.get()); + } +} + TEST(VTimestampFunctionsTest, date_format_test) { std::string func_name = "date_format"; diff --git a/conf/be.conf b/conf/be.conf index e91eb7d52d66e9..52ac34a91b6c76 100644 --- a/conf/be.conf +++ b/conf/be.conf @@ -40,7 +40,7 @@ be_port = 9060 webserver_port = 8040 heartbeat_service_port = 9050 brpc_port = 8060 -arrow_flight_port = -1 +arrow_flight_sql_port = -1 # HTTPS configures enable_https = false diff --git a/conf/fe.conf b/conf/fe.conf index 82701115b95bd5..fd145e743efe58 100644 --- a/conf/fe.conf +++ b/conf/fe.conf @@ -52,6 +52,7 @@ http_port = 8030 rpc_port = 9020 query_port = 9030 edit_log_port = 9010 +arrow_flight_sql_port = -1 # Choose one if there are more than one ip except loopback address. # Note that there should at most one ip match this list. diff --git a/docs/en/community/developer-guide/be-vscode-dev.md b/docs/en/community/developer-guide/be-vscode-dev.md index bf70f93c2cd30c..a01dabddd3b3d9 100644 --- a/docs/en/community/developer-guide/be-vscode-dev.md +++ b/docs/en/community/developer-guide/be-vscode-dev.md @@ -115,6 +115,7 @@ be_rpc_port = 9070 webserver_port = 8040 heartbeat_service_port = 9050 brpc_port = 8060 +arrow_flight_sql_port = -1 # Note that there should be at most one ip that matches this list. # If no ip matches this rule, it will choose one randomly. diff --git a/docs/en/community/developer-guide/fe-idea-dev.md b/docs/en/community/developer-guide/fe-idea-dev.md index b873bc8d153223..d52454cdb6b00e 100644 --- a/docs/en/community/developer-guide/fe-idea-dev.md +++ b/docs/en/community/developer-guide/fe-idea-dev.md @@ -174,6 +174,7 @@ sys_log_level = INFO http_port = 8030 rpc_port = 9020 query_port = 9030 +arrow_flight_sql_port = -1 edit_log_port = 9010 # Choose one if there are more than one ip except loopback address. diff --git a/docs/en/docs/admin-manual/cluster-management/upgrade.md b/docs/en/docs/admin-manual/cluster-management/upgrade.md index 41ae6baf619397..5cdb67c72aca5f 100644 --- a/docs/en/docs/admin-manual/cluster-management/upgrade.md +++ b/docs/en/docs/admin-manual/cluster-management/upgrade.md @@ -144,6 +144,7 @@ admin set frontend config("disable_tablet_scheduler" = "true"); http_port = 18030 rpc_port = 19020 query_port = 19030 + arrow_flight_sql_port = 19040 edit_log_port = 19010 ... ``` diff --git a/docs/en/docs/admin-manual/config/be-config.md b/docs/en/docs/admin-manual/config/be-config.md index 08c120674601fe..dac22a43a86ae2 100644 --- a/docs/en/docs/admin-manual/config/be-config.md +++ b/docs/en/docs/admin-manual/config/be-config.md @@ -127,6 +127,12 @@ There are two ways to configure BE configuration items: * Description: The port of BRPC on BE, used for communication between BEs * Default value: 8060 +#### `arrow_flight_sql_port` + +* Type: int32 +* Description: The port of Arrow Flight SQL server on BE, used for communication between Arrow Flight Client and BE +* Default value: -1 + #### `enable_https` * Type: bool diff --git a/docs/en/docs/admin-manual/config/fe-config.md b/docs/en/docs/admin-manual/config/fe-config.md index cdca4f048d1448..8fc84e6ef41f33 100644 --- a/docs/en/docs/admin-manual/config/fe-config.md +++ b/docs/en/docs/admin-manual/config/fe-config.md @@ -384,6 +384,12 @@ Default:9030 FE MySQL server port +#### `arrow_flight_sql_port` + +Default:-1 + +Arrow Flight SQL server port + #### `frontend_address` Status: Deprecated, not recommended use. This parameter may be deleted later diff --git a/docs/en/docs/admin-manual/http-actions/fe/bootstrap-action.md b/docs/en/docs/admin-manual/http-actions/fe/bootstrap-action.md index d08b932dd22b05..17d4dd9a5a28e5 100644 --- a/docs/en/docs/admin-manual/http-actions/fe/bootstrap-action.md +++ b/docs/en/docs/admin-manual/http-actions/fe/bootstrap-action.md @@ -76,6 +76,7 @@ none "data": { "queryPort": 9030, "rpcPort": 9020, + "arrowFlightSqlPort": 9040, "maxReplayedJournal": 17287 }, "count": 0 @@ -85,6 +86,7 @@ none * `queryPort` is the MySQL protocol port of the FE node. * `rpcPort` is the thrift RPC port of the FE node. * `maxReplayedJournal` represents the maximum metadata journal id currently played back by the FE node. + * `arrowFlightSqlPort` is the Arrow Flight SQL port of the FE node. ## Examples @@ -114,6 +116,7 @@ none "data": { "queryPort": 9030, "rpcPort": 9020, + "arrowFlightSqlPort": 9040, "maxReplayedJournal": 17287 }, "count": 0 diff --git a/docs/en/docs/admin-manual/http-actions/fe/node-action.md b/docs/en/docs/admin-manual/http-actions/fe/node-action.md index e1189b7e4e0af6..842d58c7a360e7 100644 --- a/docs/en/docs/admin-manual/http-actions/fe/node-action.md +++ b/docs/en/docs/admin-manual/http-actions/fe/node-action.md @@ -80,6 +80,7 @@ frontends: "HttpPort", "QueryPort", "RpcPort", + "ArrowFlightSqlPort", "Role", "IsMaster", "ClusterId", diff --git a/docs/en/docs/admin-manual/maint-monitor/metadata-operation.md b/docs/en/docs/admin-manual/maint-monitor/metadata-operation.md index 1975b448e5f45d..a8483da8f83915 100644 --- a/docs/en/docs/admin-manual/maint-monitor/metadata-operation.md +++ b/docs/en/docs/admin-manual/maint-monitor/metadata-operation.md @@ -239,6 +239,7 @@ FE currently has the following ports * http_port: http port, also used to push image * rpc_port: thrift server port of Frontend * query_port: Mysql connection port +* arrow_flight_sql_port: Arrow Flight SQL connection port 1. edit_log_port @@ -256,6 +257,10 @@ FE currently has the following ports After modifying the configuration, restart FE directly. This only affects mysql's connection target. +5. arrow_flight_sql_port + + After modifying the configuration, restart FE directly. This only affects arrow flight sql server connection target. + ### Recover metadata from FE memory In some extreme cases, the image file on the disk may be damaged, but the metadata in the memory is intact. At this point, we can dump the metadata from the memory and replace the image file on the disk to recover the metadata. the entire non-stop query service operation steps are as follows: diff --git a/docs/en/docs/advanced/partition/auto-partition.md b/docs/en/docs/advanced/partition/auto-partition.md new file mode 100644 index 00000000000000..c718f75454d495 --- /dev/null +++ b/docs/en/docs/advanced/partition/auto-partition.md @@ -0,0 +1,190 @@ +--- +{ + "title": "Auto Partition", + "language": "en" +} +--- + + + +# AUTO PARTITION + + + + + +The Auto Partitioning feature supports automatic detection of whether the corresponding partition exists during the data import process. If it does not exist, the partition will be created automatically and imported normally. + +## Grammer + +When building a table, use the following syntax to populate [CREATE-TABLE](../../sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE.md) with the `partition_info` section: + +1. AUTO RANGE PARTITION: + + ```SQL + AUTO PARTITION BY RANGE FUNC_CALL_EXPR + ( + ) + ``` + where + ```SQL + FUNC_CALL_EXPR ::= date_trunc ( , '' ) + ``` + +2. AUTO LIST PARTITION: + + ```SQL + AUTO PARTITION BY LIST(`partition_col`) + ( + ) + ``` + +### Usage example + +1. AUTO RANGE PARTITION + + ```SQL + CREATE TABLE `${tblDate}` ( + `TIME_STAMP` datev2 NOT NULL COMMENT 'Date of collection' + ) ENGINE=OLAP + DUPLICATE KEY(`TIME_STAMP`) + AUTO PARTITION BY RANGE date_trunc(`TIME_STAMP`, 'month') + ( + ) + DISTRIBUTED BY HASH(`TIME_STAMP`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + ``` + +2. AUTO LIST PARTITION + + ```SQL + CREATE TABLE `${tblName1}` ( + `str` varchar not null + ) ENGINE=OLAP + DUPLICATE KEY(`str`) + AUTO PARTITION BY LIST (`str`) + ( + ) + DISTRIBUTED BY HASH(`str`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + ``` + +### Using constraints + +1. Currently the AUTO RANGE PARTITION function supports only one partition column; +2. In AUTO RANGE PARTITION, the partition function supports only `date_trunc` and the partition column supports only `DATEV2` or `DATETIMEV2` format; +3. In AUTO LIST PARTITION, function calls are not supported. Partitioned columns support BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, LARGEINT, DATE, DATETIME, CHAR, VARCHAR datatypes, and partitioned values are enum values. +4. In AUTO LIST PARTITION, a separate new PARTITION is created for each fetch of a partition column for which the corresponding partition does not currently exist. + +## Sample Scenarios + +In the [Dynamic Partitioning](./dynamic-partition.md) feature, we support the automatic creation of new partitions to accommodate real-time data at specific time periods. However, in more complex scenarios, such as processing non-real-time data, the partition columns are independent of the current system time. In this case, if you need to partition the data, you need to manually organise the partitions you belong to and create them before importing the data. This is cumbersome when the number of partition columns is large. The automatic partition function solves this problem. + +For example, we have a table as follows: + +```sql +CREATE TABLE `DAILY_TRADE_VALUE` +( + `TRADE_DATE` datev2 NULL + `TRADE_ID` varchar(40) NULL, + ...... +) +UNIQUE KEY(`TRADE_DATE`, `TRADE_ID`) +PARTITION BY RANGE(`TRADE_DATE`) +( + PARTITION p_2000 VALUES [('2000-01-01'), ('2001-01-01')), + PARTITION p_2001 VALUES [('2001-01-01'), ('2002-01-01')), + PARTITION p_2002 VALUES [('2002-01-01'), ('2003-01-01')), + PARTITION p_2003 VALUES [('2003-01-01'), ('2004-01-01')), + PARTITION p_2004 VALUES [('2004-01-01'), ('2005-01-01')), + PARTITION p_2005 VALUES [('2005-01-01'), ('2006-01-01')), + PARTITION p_2006 VALUES [('2006-01-01'), ('2007-01-01')), + PARTITION p_2007 VALUES [('2007-01-01'), ('2008-01-01')), + PARTITION p_2008 VALUES [('2008-01-01'), ('2009-01-01')), + PARTITION p_2009 VALUES [('2009-01-01'), ('2010-01-01')), + PARTITION p_2010 VALUES [('2010-01-01'), ('2011-01-01')), + PARTITION p_2011 VALUES [('2011-01-01'), ('2012-01-01')), + PARTITION p_2012 VALUES [('2012-01-01'), ('2013-01-01')), + PARTITION p_2013 VALUES [('2013-01-01'), ('2014-01-01')), + PARTITION p_2014 VALUES [('2014-01-01'), ('2015-01-01')), + PARTITION p_2015 VALUES [('2015-01-01'), ('2016-01-01')), + PARTITION p_2016 VALUES [('2016-01-01'), ('2017-01-01')), + PARTITION p_2017 VALUES [('2017-01-01'), ('2018-01-01')), + PARTITION p_2018 VALUES [('2018-01-01'), ('2019-01-01')), + PARTITION p_2019 VALUES [('2019-01-01'), ('2020-01-01')), + PARTITION p_2020 VALUES [('2020-01-01'), ('2021-01-01')), + PARTITION p_2021 VALUES [('2021-01-01'), ('2022-01-01')) +) +DISTRIBUTED BY HASH(`TRADE_DATE`) BUCKETS 10 +PROPERTIES ( + "replication_num" = "1" +); +``` + +The table stores a large amount of business history data, partitioned based on the date the transaction occurred. As you can see when building the table, we need to manually create the partitions in advance. If the data range of the partitioned columns changes, for example, 2022 is added to the above table, we need to create a partition by [ALTER-TABLE-PARTITION](../../sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md) to make changes to the table partition. After using AUTO PARTITION, the table DDL can be changed to: + +```SQL +CREATE TABLE `DAILY_TRADE_VALUE` +( + `TRADE_DATE` datev2 NULL, + `TRADE_ID` varchar(40) NULL, + ...... +) +UNIQUE KEY(`TRADE_DATE`, `TRADE_ID`) +AUTO PARTITION BY RANGE date_trunc(`TRADE_DATE`, 'year') +( +) +DISTRIBUTED BY HASH(`TRADE_DATE`) BUCKETS 10 +PROPERTIES ( + "replication_num" = "1" +); +``` + +At this point the new table does not have a default partition: +```SQL +mysql> show partitions from `DAILY_TRADE_VALUE`; +Empty set (0.12 sec) +``` + +After inserting the data and then viewing it again, we could found that the table has been created with corresponding partitions: +```SQL +mysql> insert into `DAILY_TRADE_VALUE` values ('2012-12-13', 1), ('2008-02-03', 2), ('2014-11-11', 3); +Query OK, 3 rows affected (0.88 sec) +{'label':'insert_754e2a3926a345ea_854793fb2638f0ec', 'status':'VISIBLE', 'txnId':'20014'} + +mysql> show partitions from `DAILY_TRADE_VALUE`; ++-------------+-----------------+----------------+---------------------+--------+--------------+--------------------------------------------------------------------------------+-----------------+---------+----------------+---------------+---------------------+---------------------+--------------------------+----------+------------+-------------------------+-----------+ +| PartitionId | PartitionName | VisibleVersion | VisibleVersionTime | State | PartitionKey | Range | DistributionKey | Buckets | ReplicationNum | StorageMedium | CooldownTime | RemoteStoragePolicy | LastConsistencyCheckTime | DataSize | IsInMemory | ReplicaAllocation | IsMutable | ++-------------+-----------------+----------------+---------------------+--------+--------------+--------------------------------------------------------------------------------+-----------------+---------+----------------+---------------+---------------------+---------------------+--------------------------+----------+------------+-------------------------+-----------+ +| 180060 | p20080101000000 | 2 | 2023-09-18 21:49:29 | NORMAL | TRADE_DATE | [types: [DATEV2]; keys: [2008-01-01]; ..types: [DATEV2]; keys: [2009-01-01]; ) | TRADE_DATE | 10 | 1 | HDD | 9999-12-31 23:59:59 | | NULL | 0.000 | false | tag.location.default: 1 | true | +| 180039 | p20120101000000 | 2 | 2023-09-18 21:49:29 | NORMAL | TRADE_DATE | [types: [DATEV2]; keys: [2012-01-01]; ..types: [DATEV2]; keys: [2013-01-01]; ) | TRADE_DATE | 10 | 1 | HDD | 9999-12-31 23:59:59 | | NULL | 0.000 | false | tag.location.default: 1 | true | +| 180018 | p20140101000000 | 2 | 2023-09-18 21:49:29 | NORMAL | TRADE_DATE | [types: [DATEV2]; keys: [2014-01-01]; ..types: [DATEV2]; keys: [2015-01-01]; ) | TRADE_DATE | 10 | 1 | HDD | 9999-12-31 23:59:59 | | NULL | 0.000 | false | tag.location.default: 1 | true | ++-------------+-----------------+----------------+---------------------+--------+--------------+--------------------------------------------------------------------------------+-----------------+---------+----------------+---------------+---------------------+---------------------+--------------------------+----------+------------+-------------------------+-----------+ +3 rows in set (0.12 sec) +``` + +## caveat + +- If a partition is created during the insertion or importation of data and the process eventually fails, the created partition is not automatically deleted. +- Tables that use AUTO PARTITION only have their partitions created automatically instead of manually. The original use of the table and the partitions it creates is the same as for non-AUTO PARTITION tables or partitions. diff --git a/docs/en/docs/get-starting/quick-start.md b/docs/en/docs/get-starting/quick-start.md index 414fe140185d25..db145540cc7106 100644 --- a/docs/en/docs/get-starting/quick-start.md +++ b/docs/en/docs/get-starting/quick-start.md @@ -143,6 +143,7 @@ mysql> show frontends\G; HttpPort: 8030 QueryPort: 9030 RpcPort: 9020 +ArrowFlightSqlPort: 9040 Role: FOLLOWER IsMaster: true ClusterId: 1685821635 diff --git a/docs/en/docs/install/standard-deployment.md b/docs/en/docs/install/standard-deployment.md index e546d64c584930..5622ea84e1efae 100644 --- a/docs/en/docs/install/standard-deployment.md +++ b/docs/en/docs/install/standard-deployment.md @@ -123,6 +123,7 @@ Doris instances communicate directly over the network. The following table shows | FE | http_port | 8030 | FE <--> FE, user <--> FE | HTTP server port on FE | | FE | rpc_port | 9020 | BE --> FE, FE <--> FE | Thrift server port on FE; The configurations of each FE should be consistent. | | FE | query_port | 9030 | user <--> FE | MySQL server port on FE | +| FE | arrow_flight_sql_port | 9040 | user <--> FE | Arrow Flight SQL server port on FE | | FE | edit\_log_port | 9010 | FE <--> FE | Port on FE for BDBJE communication | | Broker | broker ipc_port | 8000 | FE --> Broker, BE --> Broker | Thrift server port on Broker for receiving requests | diff --git a/docs/en/docs/sql-manual/sql-functions/table-functions/frontends.md b/docs/en/docs/sql-manual/sql-functions/table-functions/frontends.md index 52bc90cd7bda65..c78f19a64987b8 100644 --- a/docs/en/docs/sql-manual/sql-functions/table-functions/frontends.md +++ b/docs/en/docs/sql-manual/sql-functions/table-functions/frontends.md @@ -56,6 +56,7 @@ mysql> desc function frontends(); | HttpPort | TEXT | No | false | NULL | NONE | | QueryPort | TEXT | No | false | NULL | NONE | | RpcPort | TEXT | No | false | NULL | NONE | +| ArrowFlightSqlPort| TEXT | No | false | NULL | NONE | | Role | TEXT | No | false | NULL | NONE | | IsMaster | TEXT | No | false | NULL | NONE | | ClusterId | TEXT | No | false | NULL | NONE | @@ -85,6 +86,7 @@ mysql> select * from frontends()\G HttpPort: 8034 QueryPort: 9033 RpcPort: 9023 +ArrowFlightSqlPort: 9040 Role: FOLLOWER IsMaster: true ClusterId: 1258341841 diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md index 03aa6968eae185..dfc7ebdd331640 100644 --- a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md +++ b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md @@ -151,7 +151,16 @@ ALTER TABLE example_db.my_table DROP PARTITION p1; ``` -8. Add a partition specifying upper and lower bounds +8. Batch delete partition + +```sql +ALTER TABLE example_db.my_table +DROP PARTITION p1, +DROP PARTITION p2, +DROP PARTITION p3; +``` + +9. Add a partition specifying upper and lower bounds ```sql ALTER TABLE example_db.my_table diff --git a/docs/sidebars.json b/docs/sidebars.json index f6b3041d49ce9e..d44dc08e89976b 100644 --- a/docs/sidebars.json +++ b/docs/sidebars.json @@ -134,7 +134,8 @@ "label": "Doris Partition", "items": [ "advanced/partition/dynamic-partition", - "advanced/partition/table-temp-partition" + "advanced/partition/table-temp-partition", + "advanced/partition/auto-partition" ] }, { diff --git a/docs/zh-CN/community/developer-guide/be-vscode-dev.md b/docs/zh-CN/community/developer-guide/be-vscode-dev.md index 7a18a186f37491..9e8a1855fa0c81 100644 --- a/docs/zh-CN/community/developer-guide/be-vscode-dev.md +++ b/docs/zh-CN/community/developer-guide/be-vscode-dev.md @@ -114,6 +114,7 @@ be_rpc_port = 9070 webserver_port = 8040 heartbeat_service_port = 9050 brpc_port = 8060 +arrow_flight_sql_port = -1 # Note that there should at most one ip match this list. # If no ip match this rule, will choose one randomly. diff --git a/docs/zh-CN/community/developer-guide/fe-idea-dev.md b/docs/zh-CN/community/developer-guide/fe-idea-dev.md index 62813adc3b5647..5eb1a70548c45a 100644 --- a/docs/zh-CN/community/developer-guide/fe-idea-dev.md +++ b/docs/zh-CN/community/developer-guide/fe-idea-dev.md @@ -169,6 +169,7 @@ sys_log_level = INFO http_port = 8030 rpc_port = 9020 query_port = 9030 +arrow_flight_sql_port = -1 edit_log_port = 9010 # Choose one if there are more than one ip except loopback address. diff --git a/docs/zh-CN/docs/admin-manual/cluster-management/upgrade.md b/docs/zh-CN/docs/admin-manual/cluster-management/upgrade.md index 27f2994b8ed434..0b2145b9c74881 100644 --- a/docs/zh-CN/docs/admin-manual/cluster-management/upgrade.md +++ b/docs/zh-CN/docs/admin-manual/cluster-management/upgrade.md @@ -144,6 +144,7 @@ admin set frontend config("disable_tablet_scheduler" = "true"); http_port = 18030 rpc_port = 19020 query_port = 19030 + arrow_flight_sql_port = 19040 edit_log_port = 19010 ... ``` diff --git a/docs/zh-CN/docs/admin-manual/config/be-config.md b/docs/zh-CN/docs/admin-manual/config/be-config.md index 7501a800199c59..9e91afd12ea20b 100644 --- a/docs/zh-CN/docs/admin-manual/config/be-config.md +++ b/docs/zh-CN/docs/admin-manual/config/be-config.md @@ -123,6 +123,12 @@ BE 重启后该配置将失效。如果想持久化修改结果,使用如下 * 描述:BE 上的 brpc 的端口,用于 BE 之间通讯 * 默认值:8060 +#### `arrow_flight_sql_port` + +* 类型:int32 +* 描述:FE 上的 Arrow Flight SQL server 的端口,用于从 Arrow Flight Client 和 BE 之间通讯 +* 默认值:-1 + #### `enable_https` * 类型:bool diff --git a/docs/zh-CN/docs/admin-manual/config/fe-config.md b/docs/zh-CN/docs/admin-manual/config/fe-config.md index 1079b604312113..e4c02ef8b468f8 100644 --- a/docs/zh-CN/docs/admin-manual/config/fe-config.md +++ b/docs/zh-CN/docs/admin-manual/config/fe-config.md @@ -384,6 +384,12 @@ heartbeat_mgr 中处理心跳事件的线程数。 Doris FE 通过 mysql 协议查询连接端口 +#### `arrow_flight_sql_port` + +默认值:-1 + +Doris FE 通过 Arrow Flight SQL 协议查询连接端口 + #### `frontend_address` 状态:已弃用,不建议使用。 diff --git a/docs/zh-CN/docs/admin-manual/http-actions/fe/bootstrap-action.md b/docs/zh-CN/docs/admin-manual/http-actions/fe/bootstrap-action.md index 689da335c984b1..d767aa1cc52620 100644 --- a/docs/zh-CN/docs/admin-manual/http-actions/fe/bootstrap-action.md +++ b/docs/zh-CN/docs/admin-manual/http-actions/fe/bootstrap-action.md @@ -76,6 +76,7 @@ under the License. "data": { "queryPort": 9030, "rpcPort": 9020, + "arrowFlightSqlPort": 9040, "maxReplayedJournal": 17287 }, "count": 0 @@ -85,6 +86,7 @@ under the License. * `queryPort` 是 FE 节点的 MySQL 协议端口。 * `rpcPort` 是 FE 节点的 thrift RPC 端口。 * `maxReplayedJournal` 表示 FE 节点当前回放的最大元数据日志id。 + * `arrowFlightSqlPort` 是 FE 节点的 Arrow Flight SQL 协议端口。 ## Examples @@ -114,6 +116,7 @@ under the License. "data": { "queryPort": 9030, "rpcPort": 9020, + "arrowFlightSqlPort": 9040, "maxReplayedJournal": 17287 }, "count": 0 diff --git a/docs/zh-CN/docs/admin-manual/http-actions/fe/node-action.md b/docs/zh-CN/docs/admin-manual/http-actions/fe/node-action.md index 9960ad1551485c..53cc693b6f986b 100644 --- a/docs/zh-CN/docs/admin-manual/http-actions/fe/node-action.md +++ b/docs/zh-CN/docs/admin-manual/http-actions/fe/node-action.md @@ -80,6 +80,7 @@ frontends: "HttpPort", "QueryPort", "RpcPort", + "ArrowFlightSqlPort", "Role", "IsMaster", "ClusterId", diff --git a/docs/zh-CN/docs/admin-manual/maint-monitor/metadata-operation.md b/docs/zh-CN/docs/admin-manual/maint-monitor/metadata-operation.md index 12ef2de434c15c..beb10e06b3c058 100644 --- a/docs/zh-CN/docs/admin-manual/maint-monitor/metadata-operation.md +++ b/docs/zh-CN/docs/admin-manual/maint-monitor/metadata-operation.md @@ -240,6 +240,7 @@ FE 目前有以下几个端口 * http_port:http 端口,也用于推送 image * rpc_port:FE 的 thrift server port * query_port:Mysql 连接端口 +* arrow_flight_sql_port: Arrow Flight SQL 连接端口 1. edit_log_port @@ -257,6 +258,9 @@ FE 目前有以下几个端口 修改配置后,直接重启 FE 即可。这个只影响到 mysql 的连接目标。 +5. arrow_flight_sql_port + + 修改配置后,直接重启 FE 即可。这个只影响到 Arrow Flight SQL 的连接目标。 ### 从 FE 内存中恢复元数据 diff --git a/docs/zh-CN/docs/advanced/partition/auto-partition.md b/docs/zh-CN/docs/advanced/partition/auto-partition.md new file mode 100644 index 00000000000000..193a0868b3728b --- /dev/null +++ b/docs/zh-CN/docs/advanced/partition/auto-partition.md @@ -0,0 +1,190 @@ +--- +{ + "title": "自动分区", + "language": "zh-CN" +} +--- + + + +# 自动分区 + + + + + +自动分区功能支持了在导入数据过程中自动检测是否存在对应所属分区。如果不存在,则会自动创建分区并正常进行导入。 + +## 语法 + +建表时,使用以下语法填充[CREATE-TABLE](../../sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-TABLE.md)时的`partition_info`部分: + +1. AUTO RANGE PARTITION: + + ```SQL + AUTO PARTITION BY RANGE FUNC_CALL_EXPR + ( + ) + ``` + 其中 + ```SQL + FUNC_CALL_EXPR ::= date_trunc ( , '' ) + ``` + +2. AUTO LIST PARTITION: + + ```SQL + AUTO PARTITION BY LIST(`partition_col`) + ( + ) + ``` + +### 用法示例 + +1. AUTO RANGE PARTITION + + ```SQL + CREATE TABLE `${tblDate}` ( + `TIME_STAMP` datev2 NOT NULL COMMENT '采集日期' + ) ENGINE=OLAP + DUPLICATE KEY(`TIME_STAMP`) + AUTO PARTITION BY RANGE date_trunc(`TIME_STAMP`, 'month') + ( + ) + DISTRIBUTED BY HASH(`TIME_STAMP`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + ``` + +2. AUTO LIST PARTITION + + ```SQL + CREATE TABLE `${tblName1}` ( + `str` varchar not null + ) ENGINE=OLAP + DUPLICATE KEY(`str`) + AUTO PARTITION BY LIST (`str`) + ( + ) + DISTRIBUTED BY HASH(`str`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + ``` + +### 约束 + +1. 当前自动分区功能仅支持一个分区列; +2. 在AUTO RANGE PARTITION中,分区函数仅支持`date_trunc`,分区列仅支持`DATEV2`或者`DATETIMEV2`格式; +3. 在AUTO LIST PARTITION中,不支持函数调用,分区列支持 BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, LARGEINT, DATE, DATETIME, CHAR, VARCHAR 数据类型,分区值为枚举值。 +4. 在AUTO LIST PARTITION中,分区列的每个当前不存在对应分区的取值,都会创建一个独立的新PARTITION。 + +## 场景示例 + +在[动态分区](./dynamic-partition.md)功能中,我们支持了按特定时间周期自动创建新分区以容纳实时数据。但在更复杂的场景下,例如处理非实时数据时,分区列与当前系统时间无关。此时如果需要进行数据分区操作,则需要用户手动整理所属分区并在数据导入前进行创建。在分区列基数较大时比较繁琐。自动分区功能解决了这一问题。 + +例如,我们有一张表如下: + +```sql +CREATE TABLE `DAILY_TRADE_VALUE` +( + `TRADE_DATE` datev2 NULL COMMENT '交易日期', + `TRADE_ID` varchar(40) NULL COMMENT '交易编号', + ...... +) +UNIQUE KEY(`TRADE_DATE`, `TRADE_ID`) +PARTITION BY RANGE(`TRADE_DATE`) +( + PARTITION p_2000 VALUES [('2000-01-01'), ('2001-01-01')), + PARTITION p_2001 VALUES [('2001-01-01'), ('2002-01-01')), + PARTITION p_2002 VALUES [('2002-01-01'), ('2003-01-01')), + PARTITION p_2003 VALUES [('2003-01-01'), ('2004-01-01')), + PARTITION p_2004 VALUES [('2004-01-01'), ('2005-01-01')), + PARTITION p_2005 VALUES [('2005-01-01'), ('2006-01-01')), + PARTITION p_2006 VALUES [('2006-01-01'), ('2007-01-01')), + PARTITION p_2007 VALUES [('2007-01-01'), ('2008-01-01')), + PARTITION p_2008 VALUES [('2008-01-01'), ('2009-01-01')), + PARTITION p_2009 VALUES [('2009-01-01'), ('2010-01-01')), + PARTITION p_2010 VALUES [('2010-01-01'), ('2011-01-01')), + PARTITION p_2011 VALUES [('2011-01-01'), ('2012-01-01')), + PARTITION p_2012 VALUES [('2012-01-01'), ('2013-01-01')), + PARTITION p_2013 VALUES [('2013-01-01'), ('2014-01-01')), + PARTITION p_2014 VALUES [('2014-01-01'), ('2015-01-01')), + PARTITION p_2015 VALUES [('2015-01-01'), ('2016-01-01')), + PARTITION p_2016 VALUES [('2016-01-01'), ('2017-01-01')), + PARTITION p_2017 VALUES [('2017-01-01'), ('2018-01-01')), + PARTITION p_2018 VALUES [('2018-01-01'), ('2019-01-01')), + PARTITION p_2019 VALUES [('2019-01-01'), ('2020-01-01')), + PARTITION p_2020 VALUES [('2020-01-01'), ('2021-01-01')), + PARTITION p_2021 VALUES [('2021-01-01'), ('2022-01-01')) +) +DISTRIBUTED BY HASH(`TRADE_DATE`) BUCKETS 10 +PROPERTIES ( + "replication_num" = "1" +); +``` + +该表内存储了大量业务历史数据,依据交易发生的日期进行分区。可以看到在建表时,我们需要预先手动创建分区。如果分区列的数据范围发生变化,例如上表中增加了2022年的数据,则我们需要通过[ALTER-TABLE-PARTITION](../../sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md)对表的分区进行更改。在使用AUTO PARTITION后,该表DDL可以改为: + +```SQL +CREATE TABLE `DAILY_TRADE_VALUE` +( + `TRADE_DATE` datev2 NULL COMMENT '交易日期', + `TRADE_ID` varchar(40) NULL COMMENT '交易编号', + ...... +) +UNIQUE KEY(`TRADE_DATE`, `TRADE_ID`) +AUTO PARTITION BY RANGE date_trunc(`TRADE_DATE`, 'year') +( +) +DISTRIBUTED BY HASH(`TRADE_DATE`) BUCKETS 10 +PROPERTIES ( + "replication_num" = "1" +); +``` + +此时新表没有默认分区: +```SQL +mysql> show partitions from `DAILY_TRADE_VALUE`; +Empty set (0.12 sec) +``` + +经过插入数据后再查看,发现该表已经创建了对应的分区: +```SQL +mysql> insert into `DAILY_TRADE_VALUE` values ('2012-12-13', 1), ('2008-02-03', 2), ('2014-11-11', 3); +Query OK, 3 rows affected (0.88 sec) +{'label':'insert_754e2a3926a345ea_854793fb2638f0ec', 'status':'VISIBLE', 'txnId':'20014'} + +mysql> show partitions from `DAILY_TRADE_VALUE`; ++-------------+-----------------+----------------+---------------------+--------+--------------+--------------------------------------------------------------------------------+-----------------+---------+----------------+---------------+---------------------+---------------------+--------------------------+----------+------------+-------------------------+-----------+ +| PartitionId | PartitionName | VisibleVersion | VisibleVersionTime | State | PartitionKey | Range | DistributionKey | Buckets | ReplicationNum | StorageMedium | CooldownTime | RemoteStoragePolicy | LastConsistencyCheckTime | DataSize | IsInMemory | ReplicaAllocation | IsMutable | ++-------------+-----------------+----------------+---------------------+--------+--------------+--------------------------------------------------------------------------------+-----------------+---------+----------------+---------------+---------------------+---------------------+--------------------------+----------+------------+-------------------------+-----------+ +| 180060 | p20080101000000 | 2 | 2023-09-18 21:49:29 | NORMAL | TRADE_DATE | [types: [DATEV2]; keys: [2008-01-01]; ..types: [DATEV2]; keys: [2009-01-01]; ) | TRADE_DATE | 10 | 1 | HDD | 9999-12-31 23:59:59 | | NULL | 0.000 | false | tag.location.default: 1 | true | +| 180039 | p20120101000000 | 2 | 2023-09-18 21:49:29 | NORMAL | TRADE_DATE | [types: [DATEV2]; keys: [2012-01-01]; ..types: [DATEV2]; keys: [2013-01-01]; ) | TRADE_DATE | 10 | 1 | HDD | 9999-12-31 23:59:59 | | NULL | 0.000 | false | tag.location.default: 1 | true | +| 180018 | p20140101000000 | 2 | 2023-09-18 21:49:29 | NORMAL | TRADE_DATE | [types: [DATEV2]; keys: [2014-01-01]; ..types: [DATEV2]; keys: [2015-01-01]; ) | TRADE_DATE | 10 | 1 | HDD | 9999-12-31 23:59:59 | | NULL | 0.000 | false | tag.location.default: 1 | true | ++-------------+-----------------+----------------+---------------------+--------+--------------+--------------------------------------------------------------------------------+-----------------+---------+----------------+---------------+---------------------+---------------------+--------------------------+----------+------------+-------------------------+-----------+ +3 rows in set (0.12 sec) +``` + +## 注意事项 + +- 在数据的插入或导入过程中如果创建了分区,而最终整个过程失败,被创建的分区不会被自动删除。 +- 使用AUTO PARTITION的表,只是分区创建方式上由手动转为了自动。表及其所创建分区的原本使用方法都与非AUTO PARTITION的表或分区相同。 diff --git a/docs/zh-CN/docs/get-starting/quick-start.md b/docs/zh-CN/docs/get-starting/quick-start.md index 8df83dab60048c..ff9e75bdfa0d0e 100644 --- a/docs/zh-CN/docs/get-starting/quick-start.md +++ b/docs/zh-CN/docs/get-starting/quick-start.md @@ -147,6 +147,7 @@ mysql> show frontends\G HttpPort: 8030 QueryPort: 9030 RpcPort: 9020 +ArrowFlightSqlPort: 9040 Role: FOLLOWER IsMaster: true ClusterId: 1685821635 @@ -277,6 +278,7 @@ mysql> SHOW BACKENDS\G BePort: 9060 HttpPort: 8040 BrpcPort: 8060 + ArrowFlightSqlPort: 8070 LastStartTime: 2022-08-16 15:31:37 LastHeartbeat: 2022-08-17 13:33:17 Alive: true diff --git a/docs/zh-CN/docs/install/standard-deployment.md b/docs/zh-CN/docs/install/standard-deployment.md index a338ab5f35f929..923dc52a3d90d1 100644 --- a/docs/zh-CN/docs/install/standard-deployment.md +++ b/docs/zh-CN/docs/install/standard-deployment.md @@ -119,6 +119,7 @@ Doris 各个实例直接通过网络进行通讯。以下表格展示了所有 | FE | http_port | 8030 | FE <--> FE,用户 <--> FE |FE 上的 http server 端口 | | FE | rpc_port | 9020 | BE --> FE, FE <--> FE | FE 上的 thrift server 端口,每个fe的配置需要保持一致| | FE | query_port | 9030 | 用户 <--> FE | FE 上的 mysql server 端口 | +| FE | arrow_flight_sql_port | 9040 | 用户 <--> FE | FE 上的 Arrow Flight SQL server 端口 | | FE | edit\_log_port | 9010 | FE <--> FE | FE 上的 bdbje 之间通信用的端口 | | Broker | broker\_ipc_port | 8000 | FE --> Broker, BE --> Broker | Broker 上的 thrift server,用于接收请求 | diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/frontends.md b/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/frontends.md index f367fd30141eb2..a2c85ec75456ba 100644 --- a/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/frontends.md +++ b/docs/zh-CN/docs/sql-manual/sql-functions/table-functions/frontends.md @@ -55,6 +55,7 @@ mysql> desc function frontends(); | HttpPort | TEXT | No | false | NULL | NONE | | QueryPort | TEXT | No | false | NULL | NONE | | RpcPort | TEXT | No | false | NULL | NONE | +| ArrowFlightSqlPort| TEXT | No | false | NULL | NONE | | Role | TEXT | No | false | NULL | NONE | | IsMaster | TEXT | No | false | NULL | NONE | | ClusterId | TEXT | No | false | NULL | NONE | @@ -84,6 +85,7 @@ mysql> select * from frontends()\G HttpPort: 8034 QueryPort: 9033 RpcPort: 9023 +ArrowFlightSqlPort: 9040 Role: FOLLOWER IsMaster: true ClusterId: 1258341841 diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md index c9ea4a4c7d32bf..4bdef84e3cf3b2 100644 --- a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Alter/ALTER-TABLE-PARTITION.md @@ -151,7 +151,16 @@ ALTER TABLE example_db.my_table DROP PARTITION p1; ``` -8. 增加一个指定上下界的分区 +8. 批量删除分区 + +```sql +ALTER TABLE example_db.my_table +DROP PARTITION p1, +DROP PARTITION p2, +DROP PARTITION p3; +``` + +9. 增加一个指定上下界的分区 ```sql ALTER TABLE example_db.my_table diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 1d4f35fef78e25..a2aa07ba24b7fe 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -390,6 +390,9 @@ public class Config extends ConfigBase { @ConfField(description = {"FE MySQL server 的端口号", "The port of FE MySQL server"}) public static int query_port = 9030; + @ConfField(description = {"FE Arrow-Flight-SQL server 的端口号", "The port of FE Arrow-Flight-SQ server"}) + public static int arrow_flight_sql_port = -1; + @ConfField(description = {"MySQL 服务的 IO 线程数", "The number of IO threads in MySQL service"}) public static int mysql_service_io_threads_num = 4; diff --git a/fe/fe-core/pom.xml b/fe/fe-core/pom.xml index 26e49d276e6432..0c00f1f4425a8d 100644 --- a/fe/fe-core/pom.xml +++ b/fe/fe-core/pom.xml @@ -496,6 +496,10 @@ under the License. org.apache.httpcomponents httpclient + + com.google.flatbuffers + flatbuffers-java + org.apache.doris hive-catalog-shade @@ -706,6 +710,64 @@ under the License. quartz 2.3.2 + + + + org.apache.arrow + arrow-memory-netty + + + io.grpc + grpc-netty + + + io.grpc + grpc-core + + + io.grpc + grpc-context + + + io.netty + netty-buffer + + + io.netty + netty-handler + + + io.netty + netty-transport + + + io.grpc + grpc-api + + + org.apache.arrow + flight-core + + + org.apache.arrow + arrow-memory-core + + + org.apache.arrow + arrow-jdbc + + + org.apache.arrow + arrow-vector + + + org.hamcrest + hamcrest + + + org.apache.arrow + flight-sql + @@ -777,7 +839,7 @@ under the License. - + de.jflex @@ -1053,5 +1115,12 @@ under the License. + + + kr.motd.maven + os-maven-plugin + 1.7.0 + + diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index a83a9dc30c6dd0..335576660f6b00 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -981,4 +981,4 @@ nonReserved | WORK | YEAR //--DEFAULT-NON-RESERVED-END - ; \ No newline at end of file + ; diff --git a/fe/fe-core/src/main/java/org/apache/doris/DorisFE.java b/fe/fe-core/src/main/java/org/apache/doris/DorisFE.java index f204d11edb5a85..59dd3d96b01a88 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/DorisFE.java +++ b/fe/fe-core/src/main/java/org/apache/doris/DorisFE.java @@ -193,7 +193,8 @@ public static void start(String dorisHomeDir, String pidDir, String[] args, Star } if (options.enableQeService) { - QeService qeService = new QeService(Config.query_port, ExecuteEnv.getInstance().getScheduler()); + QeService qeService = new QeService(Config.query_port, Config.arrow_flight_sql_port, + ExecuteEnv.getInstance().getScheduler()); qeService.start(); } @@ -231,6 +232,11 @@ private static void checkAllPorts() throws IOException { "Rpc port", NetUtils.RPC_PORT_SUGGESTION)) { throw new IOException("port " + Config.rpc_port + " already in use"); } + if (Config.arrow_flight_sql_port != -1 + && !NetUtils.isPortAvailable(FrontendOptions.getLocalHostAddress(), Config.arrow_flight_sql_port, + "Arrow Flight SQL port", NetUtils.ARROW_FLIGHT_SQL_SUGGESTION)) { + throw new IOException("port " + Config.arrow_flight_sql_port + " already in use"); + } } /* diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index 79746635492200..800788b7ca7a75 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -222,53 +222,55 @@ private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable, } else if (currentAlterOps.hasRollupOp()) { materializedViewHandler.process(alterClauses, clusterName, db, olapTable); } else if (currentAlterOps.hasPartitionOp()) { - Preconditions.checkState(alterClauses.size() == 1); - AlterClause alterClause = alterClauses.get(0); - olapTable.writeLockOrDdlException(); - try { - if (alterClause instanceof DropPartitionClause) { - if (!((DropPartitionClause) alterClause).isTempPartition()) { - DynamicPartitionUtil.checkAlterAllowed(olapTable); - } - Env.getCurrentEnv().dropPartition(db, olapTable, ((DropPartitionClause) alterClause)); - } else if (alterClause instanceof ReplacePartitionClause) { - Env.getCurrentEnv().replaceTempPartition(db, olapTable, (ReplacePartitionClause) alterClause); - } else if (alterClause instanceof ModifyPartitionClause) { - ModifyPartitionClause clause = ((ModifyPartitionClause) alterClause); - // expand the partition names if it is 'Modify Partition(*)' - if (clause.isNeedExpand()) { - List partitionNames = clause.getPartitionNames(); - partitionNames.clear(); - for (Partition partition : olapTable.getPartitions()) { - partitionNames.add(partition.getName()); + Preconditions.checkState(!alterClauses.isEmpty()); + for (AlterClause alterClause : alterClauses) { + olapTable.writeLockOrDdlException(); + try { + if (alterClause instanceof DropPartitionClause) { + if (!((DropPartitionClause) alterClause).isTempPartition()) { + DynamicPartitionUtil.checkAlterAllowed(olapTable); } - } - Map properties = clause.getProperties(); - if (properties.containsKey(PropertyAnalyzer.PROPERTIES_INMEMORY)) { - boolean isInMemory = - Boolean.parseBoolean(properties.get(PropertyAnalyzer.PROPERTIES_INMEMORY)); - if (isInMemory) { - throw new UserException("Not support set 'in_memory'='true' now!"); + Env.getCurrentEnv().dropPartition(db, olapTable, ((DropPartitionClause) alterClause)); + } else if (alterClause instanceof ReplacePartitionClause) { + Env.getCurrentEnv().replaceTempPartition(db, olapTable, (ReplacePartitionClause) alterClause); + } else if (alterClause instanceof ModifyPartitionClause) { + ModifyPartitionClause clause = ((ModifyPartitionClause) alterClause); + // expand the partition names if it is 'Modify Partition(*)' + if (clause.isNeedExpand()) { + List partitionNames = clause.getPartitionNames(); + partitionNames.clear(); + for (Partition partition : olapTable.getPartitions()) { + partitionNames.add(partition.getName()); + } } - needProcessOutsideTableLock = true; - } else { - List partitionNames = clause.getPartitionNames(); - if (!properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY)) { - modifyPartitionsProperty(db, olapTable, partitionNames, properties, - clause.isTempPartition()); - } else { + Map properties = clause.getProperties(); + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_INMEMORY)) { + boolean isInMemory = + Boolean.parseBoolean(properties.get(PropertyAnalyzer.PROPERTIES_INMEMORY)); + if (isInMemory) { + throw new UserException("Not support set 'in_memory'='true' now!"); + } needProcessOutsideTableLock = true; + } else { + List partitionNames = clause.getPartitionNames(); + if (!properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY)) { + modifyPartitionsProperty(db, olapTable, partitionNames, properties, + clause.isTempPartition()); + } else { + needProcessOutsideTableLock = true; + } } + } else if (alterClause instanceof DropPartitionFromIndexClause) { + // do nothing + } else if (alterClause instanceof AddPartitionClause + || alterClause instanceof AddPartitionLikeClause) { + needProcessOutsideTableLock = true; + } else { + throw new DdlException("Invalid alter operation: " + alterClause.getOpType()); } - } else if (alterClause instanceof DropPartitionFromIndexClause) { - // do nothing - } else if (alterClause instanceof AddPartitionClause || alterClause instanceof AddPartitionLikeClause) { - needProcessOutsideTableLock = true; - } else { - throw new DdlException("Invalid alter operation: " + alterClause.getOpType()); + } finally { + olapTable.writeUnlock(); } - } finally { - olapTable.writeUnlock(); } } else if (currentAlterOps.hasRenameOp()) { processRename(db, olapTable, alterClauses); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOpType.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOpType.java index 10377a07a77d8f..9563eaa89470a2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOpType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOpType.java @@ -62,6 +62,9 @@ public enum AlterOpType { COMPATIBILITY_MATRIX[SCHEMA_CHANGE.ordinal()][SCHEMA_CHANGE.ordinal()] = true; // can modify multi column comments at same time COMPATIBILITY_MATRIX[MODIFY_COLUMN_COMMENT.ordinal()][MODIFY_COLUMN_COMMENT.ordinal()] = true; + // can drop multi partition at same time + COMPATIBILITY_MATRIX[DROP_PARTITION.ordinal()][DROP_PARTITION.ordinal()] = true; + } public boolean needCheckCapacity() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionExprUtil.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionExprUtil.java index 750386162f6767..fcb1527eac4011 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionExprUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionExprUtil.java @@ -129,13 +129,15 @@ public static Map getAddPartitionClauseFromPartition partitionKeyDesc = createPartitionKeyDescWithRange(beginDateTime, endDateTime, partitionColumnType); } else if (partitionType == PartitionType.LIST) { List> listValues = new ArrayList<>(); - // TODO: need to support any type String pointValue = value; PartitionValue lowerValue = new PartitionValue(pointValue); listValues.add(Collections.singletonList(lowerValue)); partitionKeyDesc = PartitionKeyDesc.createIn( listValues); - partitionName += lowerValue.getStringValue(); + partitionName += getFormatPartitionValue(lowerValue.getStringValue()); + if (partitionColumnType.isStringType()) { + partitionName += "_" + System.currentTimeMillis(); + } } else { throw new AnalysisException("now only support range and list partition"); } @@ -181,6 +183,27 @@ public static PartitionKeyDesc createPartitionKeyDescWithRange(DateLiteral begin Collections.singletonList(upperValue)); } + public static String getFormatPartitionValue(String value) { + StringBuilder sb = new StringBuilder(); + // When the value is negative + if (value.length() > 0 && value.charAt(0) == '-') { + sb.append("_"); + } + for (int i = 0; i < value.length(); i++) { + char ch = value.charAt(i); + if ((ch >= 'A' && ch <= 'Z') || (ch >= 'a' && ch <= 'z') || (ch >= '0' && ch <= '9')) { + sb.append(ch); + } else if (ch == '-' || ch == ':' || ch == ' ' || ch == '*') { + // Main user remove characters in time + } else { + int unicodeValue = value.codePointAt(i); + String unicodeString = Integer.toHexString(unicodeValue); + sb.append(unicodeString); + } + } + return sb.toString(); + } + public class FunctionIntervalInfo { public String timeUnit; public int interval; diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java b/fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java index 10227ce4698f32..dbe4d5afb7a4c4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java @@ -450,7 +450,7 @@ public Status upload(String localFilePath, String remoteFilePath) { // Preconditions.checkArgument(remoteFilePath.startsWith(location), remoteFilePath); // get md5usm of local file File file = new File(localFilePath); - String md5sum = null; + String md5sum; try (FileInputStream fis = new FileInputStream(file)) { md5sum = DigestUtils.md5Hex(fis); } catch (FileNotFoundException e) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java index 4bd9cf0ffd1c55..0666d87b7176d5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java @@ -189,6 +189,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbExtractDouble; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbExtractInt; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbExtractIsnull; +import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbExtractLargeint; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbExtractString; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbParse; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbParseErrorToInvalid; @@ -202,6 +203,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbParseNullableErrorToNull; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbParseNullableErrorToValue; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbType; +import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbValid; import org.apache.doris.nereids.trees.expressions.functions.scalar.LastDay; import org.apache.doris.nereids.trees.expressions.functions.scalar.Least; import org.apache.doris.nereids.trees.expressions.functions.scalar.Left; @@ -550,25 +552,48 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(JsonQuote.class, "json_quote"), scalar(JsonUnQuote.class, "json_unquote"), scalar(JsonExtract.class, "json_extract"), + scalar(JsonbExistsPath.class, "json_exists_path"), scalar(JsonbExistsPath.class, "jsonb_exists_path"), scalar(JsonbExtract.class, "jsonb_extract"), + scalar(JsonbExtractBigint.class, "json_extract_bigint"), scalar(JsonbExtractBigint.class, "jsonb_extract_bigint"), + scalar(JsonbExtractLargeint.class, "json_extract_largeint"), + scalar(JsonbExtractLargeint.class, "jsonb_extract_largeint"), + scalar(JsonbExtractBool.class, "json_extract_bool"), scalar(JsonbExtractBool.class, "jsonb_extract_bool"), + scalar(JsonbExtractDouble.class, "json_extract_double"), scalar(JsonbExtractDouble.class, "jsonb_extract_double"), + scalar(JsonbExtractInt.class, "json_extract_int"), scalar(JsonbExtractInt.class, "jsonb_extract_int"), + scalar(JsonbExtractIsnull.class, "json_extract_isnull"), scalar(JsonbExtractIsnull.class, "jsonb_extract_isnull"), + scalar(JsonbExtractString.class, "json_extract_string"), scalar(JsonbExtractString.class, "jsonb_extract_string"), + scalar(JsonbParse.class, "json_parse"), scalar(JsonbParse.class, "jsonb_parse"), + scalar(JsonbParseErrorToInvalid.class, "json_parse_error_to_invalid"), scalar(JsonbParseErrorToInvalid.class, "jsonb_parse_error_to_invalid"), + scalar(JsonbParseErrorToNull.class, "json_parse_error_to_null"), scalar(JsonbParseErrorToNull.class, "jsonb_parse_error_to_null"), + scalar(JsonbParseErrorToValue.class, "json_parse_error_to_value"), scalar(JsonbParseErrorToValue.class, "jsonb_parse_error_to_value"), + scalar(JsonbParseNotnull.class, "json_parse_notnull"), scalar(JsonbParseNotnull.class, "jsonb_parse_notnull"), + scalar(JsonbParseNotnullErrorToInvalid.class, "json_parse_notnull_error_to_invalid"), scalar(JsonbParseNotnullErrorToInvalid.class, "jsonb_parse_notnull_error_to_invalid"), + scalar(JsonbParseNotnullErrorToValue.class, "json_parse_notnull_error_to_value"), scalar(JsonbParseNotnullErrorToValue.class, "jsonb_parse_notnull_error_to_value"), + scalar(JsonbParseNullable.class, "json_parse_nullable"), scalar(JsonbParseNullable.class, "jsonb_parse_nullable"), + scalar(JsonbParseNullableErrorToInvalid.class, "json_parse_nullable_error_to_invalid"), scalar(JsonbParseNullableErrorToInvalid.class, "jsonb_parse_nullable_error_to_invalid"), + scalar(JsonbParseNullableErrorToNull.class, "json_parse_nullable_error_to_null"), scalar(JsonbParseNullableErrorToNull.class, "jsonb_parse_nullable_error_to_null"), + scalar(JsonbParseNullableErrorToValue.class, "json_parse_nullable_error_to_value"), scalar(JsonbParseNullableErrorToValue.class, "jsonb_parse_nullable_error_to_value"), + scalar(JsonbValid.class, "json_valid"), + scalar(JsonbValid.class, "jsonb_valid"), + scalar(JsonbType.class, "json_type"), scalar(JsonbType.class, "jsonb_type"), scalar(JsonLength.class, "json_length"), scalar(JsonContains.class, "json_conatins"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java index 7658cfae0715db..647e4caf570d92 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java @@ -44,10 +44,10 @@ public class BackendsProcDir implements ProcDirInterface { private static final Logger LOG = LogManager.getLogger(BackendsProcDir.class); public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder().add("BackendId") - .add("Host").add("HeartbeatPort").add("BePort").add("HttpPort").add("BrpcPort").add("LastStartTime") - .add("LastHeartbeat").add("Alive").add("SystemDecommissioned").add("TabletNum").add("DataUsedCapacity") - .add("TrashUsedCapcacity").add("AvailCapacity").add("TotalCapacity").add("UsedPct").add("MaxDiskUsedPct") - .add("RemoteUsedCapacity").add("Tag").add("ErrMsg").add("Version").add("Status") + .add("Host").add("HeartbeatPort").add("BePort").add("HttpPort").add("BrpcPort").add("ArrowFlightSqlPort") + .add("LastStartTime").add("LastHeartbeat").add("Alive").add("SystemDecommissioned").add("TabletNum") + .add("DataUsedCapacity").add("TrashUsedCapcacity").add("AvailCapacity").add("TotalCapacity").add("UsedPct") + .add("MaxDiskUsedPct").add("RemoteUsedCapacity").add("Tag").add("ErrMsg").add("Version").add("Status") .add("HeartbeatFailureCounter").add("NodeRole") .build(); @@ -107,6 +107,7 @@ public static List> getBackendInfos() { backendInfo.add(String.valueOf(backend.getBePort())); backendInfo.add(String.valueOf(backend.getHttpPort())); backendInfo.add(String.valueOf(backend.getBrpcPort())); + backendInfo.add(String.valueOf(backend.getArrowFlightSqlPort())); backendInfo.add(TimeUtils.longToTimeString(backend.getLastStartTime())); backendInfo.add(TimeUtils.longToTimeString(backend.getLastUpdateMs())); backendInfo.add(String.valueOf(backend.isAlive())); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/FrontendsProcNode.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/FrontendsProcNode.java index e11cd81058cc2c..0e500259639c15 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/FrontendsProcNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/FrontendsProcNode.java @@ -46,7 +46,7 @@ public class FrontendsProcNode implements ProcNodeInterface { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("Name").add("Host").add("EditLogPort").add("HttpPort").add("QueryPort").add("RpcPort") - .add("Role").add("IsMaster").add("ClusterId").add("Join").add("Alive") + .add("ArrowFlightSqlPort").add("Role").add("IsMaster").add("ClusterId").add("Join").add("Alive") .add("ReplayedJournalId").add("LastStartTime").add("LastHeartbeat") .add("IsHelper").add("ErrMsg").add("Version") .add("CurrentConnected") @@ -119,9 +119,11 @@ public static void getFrontendsInfo(Env env, List> infos) { if (fe.getHost().equals(env.getSelfNode().getHost())) { info.add(Integer.toString(Config.query_port)); info.add(Integer.toString(Config.rpc_port)); + info.add(Integer.toString(Config.arrow_flight_sql_port)); } else { info.add(Integer.toString(fe.getQueryPort())); info.add(Integer.toString(fe.getRpcPort())); + info.add(Integer.toString(fe.getArrowFlightSqlPort())); } info.add(fe.getRole().name()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/NetUtils.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/NetUtils.java index 334dd11564d35f..0c1ac130cdea0a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/NetUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/NetUtils.java @@ -48,6 +48,8 @@ public class NetUtils { public static final String HTTPS_PORT_SUGGESTION = "Please change the 'https_port' in fe.conf and try again. " + "But you need to make sure that ALL FEs https_port are same."; public static final String RPC_PORT_SUGGESTION = "Please change the 'rpc_port' in fe.conf and try again."; + public static final String ARROW_FLIGHT_SQL_SUGGESTION = + "Please change the 'arrow_flight_sql_port' in fe.conf and try again."; // Target format is "host:port" public static InetSocketAddress createSocketAddr(String target) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/SmallFileMgr.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/SmallFileMgr.java index b8185b44729ba3..a95015f04b53f3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/SmallFileMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/SmallFileMgr.java @@ -418,10 +418,10 @@ public String saveToFile(long dbId, String catalog, String fileName) throws DdlE } file.createNewFile(); byte[] decoded = Base64.getDecoder().decode(smallFile.content); - FileOutputStream outputStream = new FileOutputStream(file); - outputStream.write(decoded); - outputStream.flush(); - outputStream.close(); + try (FileOutputStream outputStream = new FileOutputStream(file)) { + outputStream.write(decoded); + outputStream.flush(); + } if (!checkMd5(file, smallFile.md5)) { throw new DdlException("write file " + fileName @@ -436,7 +436,7 @@ public String saveToFile(long dbId, String catalog, String fileName) throws DdlE } private boolean checkMd5(File file, String expectedMd5) throws DdlException { - String md5sum = null; + String md5sum; try (FileInputStream fis = new FileInputStream(file)) { md5sum = DigestUtils.md5Hex(fis); } catch (FileNotFoundException e) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/EsExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/EsExternalCatalog.java index 323a31feb76a95..6f054b7946977d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/EsExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/EsExternalCatalog.java @@ -22,6 +22,7 @@ import org.apache.doris.catalog.external.EsExternalDatabase; import org.apache.doris.catalog.external.ExternalDatabase; import org.apache.doris.catalog.external.ExternalTable; +import org.apache.doris.external.elasticsearch.DorisEsException; import org.apache.doris.external.elasticsearch.EsRestClient; import com.google.common.collect.Lists; @@ -118,6 +119,10 @@ public boolean enableLikePushDown() { @Override protected void initLocalObjectsImpl() { esRestClient = new EsRestClient(getNodes(), getUsername(), getPassword(), enableSsl()); + if (!esRestClient.health()) { + throw new DorisEsException("Failed to connect to ES cluster," + + " please check your ES cluster or your ES catalog configuration."); + } } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsRestClient.java b/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsRestClient.java index c5618b8732d22c..91d3fe8954c268 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsRestClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsRestClient.java @@ -206,6 +206,11 @@ public EsShardPartitions searchShards(String indexName) throws DorisEsException return EsShardPartitions.findShardPartitions(indexName, searchShards); } + public boolean health() { + String res = execute(""); + return res != null; + } + /** * init ssl networkClient use lazy way **/ diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/BootstrapFinishAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/BootstrapFinishAction.java index b2878522edc0af..fb503f7feea11d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/BootstrapFinishAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/BootstrapFinishAction.java @@ -40,6 +40,7 @@ * "data": { * "queryPort": 9030, * "rpcPort": 9020, + * "arrowFlightSqlPort": 9040, * "maxReplayedJournal": 17287 * }, * "count": 0 @@ -53,6 +54,7 @@ public class BootstrapFinishAction extends RestBaseController { public static final String REPLAYED_JOURNAL_ID = "replayedJournalId"; public static final String QUERY_PORT = "queryPort"; + public static final String ARROW_FLIGHT_SQL_PORT = "arrowFlightSqlPort"; public static final String RPC_PORT = "rpcPort"; public static final String VERSION = "version"; @@ -91,6 +93,7 @@ public ResponseEntity execute(HttpServletRequest request, HttpServletResponse re result.setReplayedJournalId(replayedJournalId); result.setQueryPort(Config.query_port); result.setRpcPort(Config.rpc_port); + result.setArrowFlightSqlPort(Config.arrow_flight_sql_port); result.setVersion(Version.DORIS_BUILD_VERSION + "-" + Version.DORIS_BUILD_SHORT_HASH); } @@ -107,6 +110,7 @@ private static class BootstrapResult { private long replayedJournalId = 0; private int queryPort = 0; private int rpcPort = 0; + private int arrowFlightSqlPort = 0; private String version = ""; public BootstrapResult() { @@ -125,10 +129,18 @@ public void setQueryPort(int queryPort) { this.queryPort = queryPort; } + public void setArrowFlightSqlPort(int arrowFlightSqlPort) { + this.arrowFlightSqlPort = arrowFlightSqlPort; + } + public int getQueryPort() { return queryPort; } + public int getArrowFlightSqlPort() { + return arrowFlightSqlPort; + } + public void setRpcPort(int rpcPort) { this.rpcPort = rpcPort; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/manager/ClusterAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/manager/ClusterAction.java index 983bafc8522419..929e46101321cf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/manager/ClusterAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/manager/ClusterAction.java @@ -65,6 +65,8 @@ public Object clusterInfo(HttpServletRequest request, HttpServletResponse respon result.put("mysql", frontends.stream().map(ip -> ip + ":" + Config.query_port).collect(Collectors.toList())); result.put("http", frontends.stream().map(ip -> ip + ":" + Config.http_port).collect(Collectors.toList())); + result.put("arrow flight sql server", frontends.stream().map( + ip -> ip + ":" + Config.arrow_flight_sql_port).collect(Collectors.toList())); return ResponseEntityBuilder.ok(result); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/bdbje/BDBDebugger.java b/fe/fe-core/src/main/java/org/apache/doris/journal/bdbje/BDBDebugger.java index e6b0f872f4be19..ae48526515af88 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/journal/bdbje/BDBDebugger.java +++ b/fe/fe-core/src/main/java/org/apache/doris/journal/bdbje/BDBDebugger.java @@ -87,7 +87,8 @@ private void startService(String dorisHomeDir) throws Exception { httpServer.start(); // MySQl server - QeService qeService = new QeService(Config.query_port, ExecuteEnv.getInstance().getScheduler()); + QeService qeService = new QeService(Config.query_port, Config.arrow_flight_sql_port, + ExecuteEnv.getInstance().getScheduler()); qeService.start(); ThreadPoolManager.registerAllThreadPoolMetric(); 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 a65ee96fab76b8..7cbeb589195961 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 @@ -178,6 +178,15 @@ public void plan(TUniqueId loadId, List> fileStatusesLis } } + // analyze expr in whereExpr before rewrite + scanTupleDesc.setTable(table); + analyzer.registerTupleDescriptor(scanTupleDesc); + for (BrokerFileGroup fileGroup : fileGroups) { + if (fileGroup.getWhereExpr() != null) { + fileGroup.getWhereExpr().analyze(analyzer); + } + } + // Generate plan trees // 1. Broker scan node ScanNode scanNode; diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkRepository.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkRepository.java index d916c624c46580..bfdad97aa0f2ac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkRepository.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkRepository.java @@ -252,7 +252,7 @@ private void getLibraries(String remoteArchivePath, List libraries public String getMd5String(String filePath) throws LoadException { File file = new File(filePath); - String md5sum = null; + String md5sum; try (FileInputStream fis = new FileInputStream(file)) { md5sum = DigestUtils.md5Hex(fis); Preconditions.checkNotNull(md5sum); diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/MetaHelper.java b/fe/fe-core/src/main/java/org/apache/doris/master/MetaHelper.java index e1de241058387c..fe516b02bfd360 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/MetaHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/MetaHelper.java @@ -126,8 +126,7 @@ public static void getRemoteFile(String urlStr, int timeout, File file) public static String getResponse(HttpURLConnection conn) throws IOException { String response; - try (BufferedReader bufferedReader = new BufferedReader( - new InputStreamReader(conn.getInputStream()))) { + try (BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(conn.getInputStream()))) { String line; StringBuilder sb = new StringBuilder(); while ((line = bufferedReader.readLine()) != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 5db8d54d2b8dbf..1982023cc5130f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -333,7 +333,8 @@ public PlanFragment visitPhysicalDistribute(PhysicalDistribute d public PlanFragment visitPhysicalResultSink(PhysicalResultSink physicalResultSink, PlanTranslatorContext context) { PlanFragment planFragment = physicalResultSink.child().accept(this, context); - planFragment.setSink(new ResultSink(planFragment.getPlanRoot().getId())); + planFragment.setSink(new ResultSink(planFragment.getPlanRoot().getId(), + ConnectContext.get().getResultSinkType())); return planFragment; } @@ -353,29 +354,14 @@ public PlanFragment visitPhysicalOlapTableSink(PhysicalOlapTableSink targetTableColumns = olapTableSink.getTargetTable().getFullSchema(); - for (Column column : targetTableColumns) { - SlotDescriptor slotDesc = context.addSlotDesc(olapTuple); - slotDesc.setIsMaterialized(true); - slotDesc.setType(column.getType()); - slotDesc.setColumn(column); - slotDesc.setIsNullable(column.isAllowNull()); - } - OlapTableSink sink = new OlapTableSink( - olapTableSink.getTargetTable(), - olapTuple, - olapTableSink.getPartitionIds().isEmpty() ? null : olapTableSink.getPartitionIds(), - olapTableSink.isSingleReplicaLoad() - ); - if (olapTableSink.isPartialUpdate() || (olapTableSink.isFromNativeInsertStmt() - && ConnectContext.get().getSessionVariable().isEnableUniqueKeyPartialUpdate())) { - OlapTable olapTable = olapTableSink.getTargetTable(); + HashSet partialUpdateCols = new HashSet<>(); + boolean isPartialUpdate = olapTableSink.isPartialUpdate(); + if (isPartialUpdate) { + OlapTable olapTable = (OlapTable) olapTableSink.getTargetTable(); if (!olapTable.getEnableUniqueKeyMergeOnWrite()) { throw new AnalysisException("Partial update is only allowed in" + "unique table with merge-on-write enabled."); } - HashSet partialUpdateCols = new HashSet<>(); for (Column col : olapTable.getFullSchema()) { boolean exists = false; for (Column insertCol : olapTableSink.getCols()) { @@ -396,8 +382,26 @@ public PlanFragment visitPhysicalOlapTableSink(PhysicalOlapTableSink targetTableColumns = olapTableSink.getTargetTable().getFullSchema(); + for (Column column : targetTableColumns) { + if (isPartialUpdate && !partialUpdateCols.contains(column.getName())) { + continue; + } + SlotDescriptor slotDesc = context.addSlotDesc(olapTuple); + slotDesc.setIsMaterialized(true); + slotDesc.setType(column.getType()); + slotDesc.setColumn(column); + slotDesc.setIsNullable(column.isAllowNull()); + } + OlapTableSink sink = new OlapTableSink( + olapTableSink.getTargetTable(), + olapTuple, + olapTableSink.getPartitionIds().isEmpty() ? null : olapTableSink.getPartitionIds(), + olapTableSink.isSingleReplicaLoad() + ); + sink.setPartialUpdateInputColumns(isPartialUpdate, partialUpdateCols); rootFragment.setSink(sink); return rootFragment; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/DeriveStatsJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/DeriveStatsJob.java index cfe952c0f26873..572b07a1348a0d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/DeriveStatsJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/DeriveStatsJob.java @@ -28,6 +28,7 @@ import org.apache.doris.nereids.metrics.event.StatsStateEvent; import org.apache.doris.nereids.stats.StatsCalculator; import org.apache.doris.nereids.trees.expressions.CTEId; +import org.apache.doris.nereids.trees.plans.algebra.Project; import org.apache.doris.qe.ConnectContext; import org.apache.doris.statistics.Statistics; @@ -116,6 +117,17 @@ public void execute() { context.getCascadesContext().getConnectContext().getTotalHistogramMap() .putAll(statsCalculator.getTotalHistogramMap()); } + + if (groupExpression.getPlan() instanceof Project) { + // In the context of reorder join, when a new plan is generated, it may include a project operation. + // In this case, the newly generated join root and the original join root will no longer be in the + // same group. To avoid inconsistencies in the statistics between these two groups, we keep the + // child group's row count unchanged when the parent group expression is a project operation. + double parentRowCount = groupExpression.getOwnerGroup().getStatistics().getRowCount(); + groupExpression.children().forEach(g -> g.setStatistics( + g.getStatistics().updateRowCountOnly(parentRowCount)) + ); + } } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 5183b0c4b0456d..a9b50c8bf1ebc1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -411,7 +411,7 @@ public LogicalPlan visitInsertIntoQuery(InsertIntoQueryContext ctx) { colNames, ImmutableList.of(), partitions, - false, + ConnectContext.get().getSessionVariable().isEnableUniqueKeyPartialUpdate(), true, visitQuery(ctx.query())); if (ctx.explain() != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java index 7632a10f5d2071..b8400caaebdd86 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java @@ -142,7 +142,14 @@ public List buildRules() { .filter(col -> col.getName().equals(table.getSequenceMapCol())) .findFirst().get(); columnToOutput.put(column.getName(), columnToOutput.get(seqCol.getName())); + } else if (sink.isPartialUpdate()) { + // If the current load is a partial update, the values of unmentioned + // columns will be filled in SegmentWriter. And the output of sink node + // should not contain these unmentioned columns, so we just skip them. + continue; } else if (column.getDefaultValue() == null) { + // Otherwise, the unmentioned columns should be filled with default values + // or null values columnToOutput.put(column.getName(), new Alias( new NullLiteral(DataType.fromCatalogType(column.getType())), column.getName() @@ -191,8 +198,17 @@ public List buildRules() { // add cast project List castExprs = Lists.newArrayList(); for (int i = 0; i < table.getFullSchema().size(); ++i) { - Expression castExpr = TypeCoercionUtils.castIfNotSameType(fullOutputExprs.get(i), - DataType.fromCatalogType(table.getFullSchema().get(i).getType())); + Column col = table.getFullSchema().get(i); + NamedExpression expr = (NamedExpression) columnToOutput.get(col.getName()); + if (expr == null) { + // If `expr` is null, it means that the current load is a partial update + // and `col` should not be contained in the output of the sink node so + // we skip it. + continue; + } + Expression castExpr = TypeCoercionUtils.castIfNotSameType( + expr, + DataType.fromCatalogType(col.getType())); if (castExpr instanceof NamedExpression) { castExprs.add(((NamedExpression) castExpr)); } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/check/CheckCast.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/check/CheckCast.java index 84abcb9d43c55d..fe25bb8e7b9f94 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/check/CheckCast.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/check/CheckCast.java @@ -24,6 +24,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.JsonType; import org.apache.doris.nereids.types.MapType; import org.apache.doris.nereids.types.StructField; import org.apache.doris.nereids.types.StructType; @@ -69,6 +70,8 @@ private boolean check(DataType originalType, DataType targetType) { return true; } else if (originalType instanceof CharacterType && targetType instanceof StructType) { return true; + } else if (originalType instanceof JsonType || targetType instanceof JsonType) { + return true; } else { return checkPrimitiveType(originalType, targetType); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckDataTypes.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckDataTypes.java index 8f1741167b1850..d25786849192dd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckDataTypes.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckDataTypes.java @@ -27,7 +27,6 @@ import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.DataType; -import org.apache.doris.nereids.types.JsonType; import org.apache.doris.nereids.types.MapType; import org.apache.doris.nereids.types.StructType; import org.apache.doris.nereids.types.UnsupportedType; @@ -42,7 +41,7 @@ public class CheckDataTypes implements CustomRewriter { private static final Set> UNSUPPORTED_TYPE = ImmutableSet.of( - JsonType.class, UnsupportedType.class); + UnsupportedType.class); @Override public Plan rewriteRoot(Plan rootPlan, JobContext jobContext) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayUnion.java index a1152b2c6dcacc..67efc5d3811529 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayUnion.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ArrayUnion.java @@ -25,6 +25,7 @@ import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.coercion.AnyDataType; +import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -39,14 +40,14 @@ public class ArrayUnion extends ScalarFunction implements ExplicitlyCastableSign public static final List SIGNATURES = ImmutableList.of( FunctionSignature.retArgType(0) - .args(ArrayType.of(new AnyDataType(0)), ArrayType.of(new AnyDataType(0))) + .varArgs(ArrayType.of(new AnyDataType(0)), ArrayType.of(new AnyDataType(0))) ); /** - * constructor with 2 arguments. + * constructor with more than 2 arguments. */ - public ArrayUnion(Expression arg0, Expression arg1) { - super("array_union", arg0, arg1); + public ArrayUnion(Expression arg0, Expression arg1, Expression ...varArgs) { + super("array_union", ExpressionUtils.mergeArguments(arg0, arg1, varArgs)); } /** @@ -54,8 +55,9 @@ public ArrayUnion(Expression arg0, Expression arg1) { */ @Override public ArrayUnion withChildren(List children) { - Preconditions.checkArgument(children.size() == 2); - return new ArrayUnion(children.get(0), children.get(1)); + Preconditions.checkArgument(children.size() >= 2); + return new ArrayUnion(children.get(0), children.get(1), + children.subList(2, children.size()).toArray(new Expression[0])); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/JsonbExtract.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/JsonbExtract.java index bbf16255f34936..6e251bc0ee1dc4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/JsonbExtract.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/JsonbExtract.java @@ -26,6 +26,7 @@ import org.apache.doris.nereids.types.JsonType; import org.apache.doris.nereids.types.StringType; import org.apache.doris.nereids.types.VarcharType; +import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -39,15 +40,15 @@ public class JsonbExtract extends ScalarFunction implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable { public static final List SIGNATURES = ImmutableList.of( - FunctionSignature.ret(JsonType.INSTANCE).args(JsonType.INSTANCE, VarcharType.SYSTEM_DEFAULT), - FunctionSignature.ret(JsonType.INSTANCE).args(JsonType.INSTANCE, StringType.INSTANCE) + FunctionSignature.ret(JsonType.INSTANCE).varArgs(JsonType.INSTANCE, VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(JsonType.INSTANCE).varArgs(JsonType.INSTANCE, StringType.INSTANCE) ); /** - * constructor with 2 arguments. + * constructor with 2 or more arguments. */ - public JsonbExtract(Expression arg0, Expression arg1) { - super("jsonb_extract", arg0, arg1); + public JsonbExtract(Expression arg0, Expression arg1, Expression... varArgs) { + super("jsonb_extract", ExpressionUtils.mergeArguments(arg0, arg1, varArgs)); } /** @@ -55,8 +56,9 @@ public JsonbExtract(Expression arg0, Expression arg1) { */ @Override public JsonbExtract withChildren(List children) { - Preconditions.checkArgument(children.size() == 2); - return new JsonbExtract(children.get(0), children.get(1)); + Preconditions.checkArgument(children.size() >= 2, "JsonbExtract should have at least two arguments"); + return new JsonbExtract(children.get(0), children.get(1), + children.subList(2, children.size()).toArray(new Expression[0])); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/JsonbValid.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/JsonbValid.java new file mode 100644 index 00000000000000..220f3eeeb3168f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/JsonbValid.java @@ -0,0 +1,70 @@ +// 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.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.IntegerType; +import org.apache.doris.nereids.types.VarcharType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'jsonb_valid'. This class is generated by GenerateFunction. + */ +public class JsonbValid extends ScalarFunction + implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(IntegerType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT) + ); + + /** + * constructor with 1 arguments. + */ + public JsonbValid(Expression arg0) { + super("json_valid", arg0); + } + + /** + * withChildren. + */ + @Override + public JsonbValid withChildren(List children) { + Preconditions.checkArgument(children.size() == 1); + return new JsonbValid(children.get(0)); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitJsonbValid(this, context); + } +} + diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java index 575025bedf3922..c15f03f2bfc71c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java @@ -203,6 +203,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbParseNullableErrorToNull; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbParseNullableErrorToValue; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbType; +import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbValid; import org.apache.doris.nereids.trees.expressions.functions.scalar.LastDay; import org.apache.doris.nereids.trees.expressions.functions.scalar.Least; import org.apache.doris.nereids.trees.expressions.functions.scalar.Left; @@ -1149,6 +1150,10 @@ default R visitJsonContains(JsonContains jsonContains, C context) { return visitScalarFunction(jsonContains, context); } + default R visitJsonbValid(JsonbValid jsonbValid, C context) { + return visitScalarFunction(jsonbValid, context); + } + default R visitLastDay(LastDay lastDay, C context) { return visitScalarFunction(lastDay, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java index 9d47aa2ecb21d8..e6affcb545ffda 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java @@ -146,11 +146,16 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { physicalOlapTableSink.getDatabase(), physicalOlapTableSink.getTargetTable(), label, planner); isTxnBegin = true; - + boolean isStrictMode = (ctx.getSessionVariable().getEnableInsertStrict() + && physicalOlapTableSink.isPartialUpdate() + && physicalOlapTableSink.isFromNativeInsertStmt()); sink.init(ctx.queryId(), txn.getTxnId(), physicalOlapTableSink.getDatabase().getId(), ctx.getExecTimeout(), - ctx.getSessionVariable().getSendBatchParallelism(), false, false, false); + ctx.getSessionVariable().getSendBatchParallelism(), + false, + isStrictMode, + false); sink.complete(new Analyzer(Env.getCurrentEnv(), ctx)); TransactionState state = Env.getCurrentGlobalTransactionMgr().getTransactionState( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/JsonType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/JsonType.java index 8455daaac26f58..ce5ebe8658ccdb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/JsonType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/JsonType.java @@ -19,6 +19,7 @@ import org.apache.doris.catalog.Type; import org.apache.doris.nereids.annotation.Developing; +import org.apache.doris.nereids.types.coercion.CharacterType; /** * Json type in Nereids. @@ -40,7 +41,7 @@ public Type toCatalogDataType() { @Override public boolean acceptsType(DataType other) { - return other instanceof JsonType; + return other instanceof JsonType || other instanceof CharacterType; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index de11eba5fdab97..25d6bd089dd937 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -95,7 +95,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.io.File; import java.io.IOException; import java.util.List; @@ -1157,12 +1156,6 @@ public synchronized void close() throws IOException { journal.close(); } - public synchronized void createEditLogFile(File name) throws IOException { - EditLogOutputStream editLogOutputStream = new EditLogFileOutputStream(name); - editLogOutputStream.create(); - editLogOutputStream.close(); - } - public void open() { journal.open(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/Storage.java b/fe/fe-core/src/main/java/org/apache/doris/persist/Storage.java index d8fc041affc3fc..f0583722278655 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/Storage.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/Storage.java @@ -96,9 +96,9 @@ public void reload() throws IOException { Properties prop = new Properties(); File versionFile = getVersionFile(); if (versionFile.isFile()) { - FileInputStream in = new FileInputStream(versionFile); - prop.load(in); - in.close(); + try (FileInputStream in = new FileInputStream(versionFile)) { + prop.load(in); + } clusterID = Integer.parseInt(prop.getProperty(CLUSTER_ID)); if (prop.getProperty(TOKEN) != null) { token = prop.getProperty(TOKEN); @@ -107,9 +107,9 @@ public void reload() throws IOException { File roleFile = getRoleFile(); if (roleFile.isFile()) { - FileInputStream in = new FileInputStream(roleFile); - prop.load(in); - in.close(); + try (FileInputStream in = new FileInputStream(roleFile)) { + prop.load(in); + } role = FrontendNodeType.valueOf(prop.getProperty(FRONTEND_ROLE)); // For compatibility, NODE_NAME may not exist in ROLE file, set nodeName to null nodeName = prop.getProperty(NODE_NAME, null); diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java index a719081496b05b..7dc45029e7e509 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java @@ -268,6 +268,7 @@ private PlanFragment createMergeFragment(PlanFragment inputFragment) mergePlan.init(ctx.getRootAnalyzer()); Preconditions.checkState(mergePlan.hasValidStats()); PlanFragment fragment = new PlanFragment(ctx.getNextFragmentId(), mergePlan, DataPartition.UNPARTITIONED); + fragment.setResultSinkType(ctx.getRootAnalyzer().getContext().getResultSinkType()); inputFragment.setDestination(mergePlan); return fragment; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/PlanFragment.java b/fe/fe-core/src/main/java/org/apache/doris/planner/PlanFragment.java index 3d74bfc0dfc3d9..16be7e17a09810 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/PlanFragment.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/PlanFragment.java @@ -31,6 +31,7 @@ import org.apache.doris.thrift.TExplainLevel; import org.apache.doris.thrift.TPartitionType; import org.apache.doris.thrift.TPlanFragment; +import org.apache.doris.thrift.TResultSinkType; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; @@ -145,6 +146,8 @@ public class PlanFragment extends TreeNode { // has colocate plan node private boolean hasColocatePlanNode = false; + private TResultSinkType resultSinkType = TResultSinkType.MYSQL_PROTOCAL; + /** * C'tor for fragment with specific partition; the output is by default broadcast. */ @@ -234,6 +237,10 @@ public void setHasColocatePlanNode(boolean hasColocatePlanNode) { this.hasColocatePlanNode = hasColocatePlanNode; } + public void setResultSinkType(TResultSinkType resultSinkType) { + this.resultSinkType = resultSinkType; + } + public boolean hasColocatePlanNode() { return hasColocatePlanNode; } @@ -269,7 +276,7 @@ public void finalize(StatementBase stmtBase) { } else { // add ResultSink // we're streaming to an result sink - sink = new ResultSink(planRoot.getId()); + sink = new ResultSink(planRoot.getId(), resultSinkType); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/ResultSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/ResultSink.java index 49a4ca3333aa11..1b0b745223caa9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/ResultSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/ResultSink.java @@ -22,6 +22,7 @@ import org.apache.doris.thrift.TExplainLevel; import org.apache.doris.thrift.TFetchOption; import org.apache.doris.thrift.TResultSink; +import org.apache.doris.thrift.TResultSinkType; /** * Result sink that forwards data to @@ -33,10 +34,17 @@ public class ResultSink extends DataSink { // Two phase fetch option private TFetchOption fetchOption; + private TResultSinkType resultSinkType = TResultSinkType.MYSQL_PROTOCAL; + public ResultSink(PlanNodeId exchNodeId) { this.exchNodeId = exchNodeId; } + public ResultSink(PlanNodeId exchNodeId, TResultSinkType resultSinkType) { + this.exchNodeId = exchNodeId; + this.resultSinkType = resultSinkType; + } + @Override public String getExplainString(String prefix, TExplainLevel explainLevel) { StringBuilder strBuilder = new StringBuilder(); @@ -49,6 +57,7 @@ public String getExplainString(String prefix, TExplainLevel explainLevel) { strBuilder.append(prefix).append(" ").append("FETCH ROW STORE\n"); } } + strBuilder.append(prefix).append(" ").append(resultSinkType).append("\n"); return strBuilder.toString(); } @@ -63,6 +72,7 @@ protected TDataSink toThrift() { if (fetchOption != null) { tResultSink.setFetchOption(fetchOption); } + tResultSink.setType(resultSinkType); result.setResultSink(tResultSink); return result; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileGroupInfo.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileGroupInfo.java index 80fd9013457a92..2b412da8253314 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileGroupInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/FileGroupInfo.java @@ -32,6 +32,7 @@ import org.apache.doris.system.Backend; import org.apache.doris.thrift.TBrokerFileStatus; import org.apache.doris.thrift.TExternalScanRange; +import org.apache.doris.thrift.TFileCompressType; import org.apache.doris.thrift.TFileFormatType; import org.apache.doris.thrift.TFileRangeDesc; import org.apache.doris.thrift.TFileScanRange; @@ -206,16 +207,17 @@ public void createScanRangeLocations(FileLoadScanNode.ParamCreateContext context // header_type TFileFormatType formatType = formatType(context.fileGroup.getFileFormat(), fileStatus.path); context.params.setFormatType(formatType); - context.params.setCompressType( - Util.getOrInferCompressType(context.fileGroup.getCompressType(), fileStatus.path) - ); + TFileCompressType compressType = + Util.getOrInferCompressType(context.fileGroup.getCompressType(), fileStatus.path); + context.params.setCompressType(compressType); List columnsFromPath = BrokerUtil.parseColumnsFromPath(fileStatus.path, context.fileGroup.getColumnNamesFromPath()); // Assign scan range locations only for broker load. // stream load has only one file, and no need to set multi scan ranges. if (tmpBytes > bytesPerInstance && jobType != JobType.STREAM_LOAD) { // Now only support split plain text - if ((formatType == TFileFormatType.FORMAT_CSV_PLAIN && fileStatus.isSplitable) + if (compressType == TFileCompressType.PLAIN + && (formatType == TFileFormatType.FORMAT_CSV_PLAIN && fileStatus.isSplitable) || formatType == TFileFormatType.FORMAT_JSON) { long rangeBytes = bytesPerInstance - curInstanceBytes; TFileRangeDesc rangeDesc = createFileRangeDesc(curFileOffset, fileStatus, rangeBytes, @@ -223,10 +225,9 @@ public void createScanRangeLocations(FileLoadScanNode.ParamCreateContext context curLocations.getScanRange().getExtScanRange().getFileScanRange().addToRanges(rangeDesc); curFileOffset += rangeBytes; } else { - TFileRangeDesc rangeDesc = createFileRangeDesc(curFileOffset, fileStatus, leftBytes, + TFileRangeDesc rangeDesc = createFileRangeDesc(0, fileStatus, leftBytes, columnsFromPath); curLocations.getScanRange().getExtScanRange().getFileScanRange().addToRanges(rangeDesc); - curFileOffset = 0; i++; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java index bf7c84a2c4db6c..5a427a108010ec 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java @@ -42,6 +42,7 @@ import org.apache.doris.statistics.Histogram; import org.apache.doris.system.Backend; import org.apache.doris.task.LoadTaskInfo; +import org.apache.doris.thrift.TResultSinkType; import org.apache.doris.thrift.TUniqueId; import org.apache.doris.transaction.TransactionEntry; import org.apache.doris.transaction.TransactionStatus; @@ -176,6 +177,8 @@ public class ConnectContext { private String workloadGroupName = ""; private Map insertGroupCommitTableToBeMap = new HashMap<>(); + private TResultSinkType resultSinkType = TResultSinkType.MYSQL_PROTOCAL; + public void setUserQueryTimeout(int queryTimeout) { if (queryTimeout > 0) { sessionVariable.setQueryTimeoutS(queryTimeout); @@ -221,6 +224,10 @@ public MysqlSslContext getMysqlSslContext() { return mysqlSslContext; } + public TResultSinkType getResultSinkType() { + return resultSinkType; + } + public void setOrUpdateInsertResult(long txnId, String label, String db, String tbl, TransactionStatus txnStatus, long loadedRows, int filteredRows) { if (isTxnModel() && insertResult != null) { @@ -644,6 +651,10 @@ public void setStatementContext(StatementContext statementContext) { this.statementContext = statementContext; } + public void setResultSinkType(TResultSinkType resultSinkType) { + this.resultSinkType = resultSinkType; + } + // kill operation with no protect. public void kill(boolean killConnection) { LOG.warn("kill query from {}, kill connection: {}", getMysqlChannel().getRemoteHostPortString(), diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 48f71915310603..feff5b7ebec69e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -19,6 +19,7 @@ import org.apache.doris.analysis.Analyzer; import org.apache.doris.analysis.DescriptorTable; +import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.StorageBackend; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.FsBroker; @@ -207,6 +208,11 @@ public class Coordinator implements CoordInterface { private final List needCheckBackendExecStates = Lists.newArrayList(); private final List needCheckPipelineExecContexts = Lists.newArrayList(); private ResultReceiver receiver; + private TNetworkAddress resultFlightServerAddr; + private TNetworkAddress resultInternalServiceAddr; + private ArrayList resultOutputExprs; + + private TUniqueId finstId; private final List scanNodes; private int scanRangeNum = 0; // number of instances of this query, equals to @@ -274,6 +280,22 @@ public ExecutionProfile getExecutionProfile() { return executionProfile; } + public TNetworkAddress getResultFlightServerAddr() { + return resultFlightServerAddr; + } + + public TNetworkAddress getResultInternalServiceAddr() { + return resultInternalServiceAddr; + } + + public ArrayList getResultOutputExprs() { + return resultOutputExprs; + } + + public TUniqueId getFinstId() { + return finstId; + } + // True if all scan node are ExternalScanNode. private boolean isAllExternalScan = true; @@ -598,6 +620,10 @@ public void exec() throws Exception { TNetworkAddress execBeAddr = topParams.instanceExecParams.get(0).host; receiver = new ResultReceiver(queryId, topParams.instanceExecParams.get(0).instanceId, addressToBackendID.get(execBeAddr), toBrpcHost(execBeAddr), this.timeoutDeadline); + finstId = topParams.instanceExecParams.get(0).instanceId; + resultFlightServerAddr = toArrowFlightHost(execBeAddr); + resultInternalServiceAddr = toBrpcHost(execBeAddr); + resultOutputExprs = fragments.get(0).getOutputExprs(); if (LOG.isDebugEnabled()) { LOG.debug("dispatch query job: {} to {}", DebugUtil.printId(queryId), topParams.instanceExecParams.get(0).host); @@ -1595,6 +1621,18 @@ private TNetworkAddress toBrpcHost(TNetworkAddress host) throws Exception { return new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); } + private TNetworkAddress toArrowFlightHost(TNetworkAddress host) throws Exception { + Backend backend = Env.getCurrentSystemInfo().getBackendWithBePort( + host.getHostname(), host.getPort()); + if (backend == null) { + throw new UserException(SystemInfoService.NO_BACKEND_LOAD_AVAILABLE_MSG); + } + if (backend.getArrowFlightSqlPort() < 0) { + return null; + } + return new TNetworkAddress(backend.getHost(), backend.getArrowFlightSqlPort()); + } + // estimate if this fragment contains UnionNode private boolean containsUnionNode(PlanNode node) { if (node instanceof UnionNode) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/HelpModule.java b/fe/fe-core/src/main/java/org/apache/doris/qe/HelpModule.java index adca85a2a2c26e..5658c11ce1cfc1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/HelpModule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/HelpModule.java @@ -104,12 +104,12 @@ public void setUpByZip(String path) throws IOException, UserException { String line; List lines = Lists.newArrayList(); if (size > 0) { - BufferedReader reader = new BufferedReader(new InputStreamReader(zf.getInputStream(entry), - CHARSET_UTF_8)); - while ((line = reader.readLine()) != null) { - lines.add(line); + try (BufferedReader reader = + new BufferedReader(new InputStreamReader(zf.getInputStream(entry), CHARSET_UTF_8))) { + while ((line = reader.readLine()) != null) { + lines.add(line); + } } - reader.close(); // note that we only need basename String parentPathStr = null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/PointQueryExec.java b/fe/fe-core/src/main/java/org/apache/doris/qe/PointQueryExec.java index 0ffb5b989d8370..50c422a1979340 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/PointQueryExec.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/PointQueryExec.java @@ -247,10 +247,10 @@ private RowBatch getNextInternal(Status status, Backend backend) throws TExcepti while (pResult == null) { InternalService.PTabletKeyLookupRequest request = requestBuilder.build(); Future futureResponse = - BackendServiceProxy.getInstance().fetchTabletDataAsync(backend.getBrpcAdress(), request); + BackendServiceProxy.getInstance().fetchTabletDataAsync(backend.getBrpcAddress(), request); long currentTs = System.currentTimeMillis(); if (currentTs >= timeoutTs) { - LOG.warn("fetch result timeout {}", backend.getBrpcAdress()); + LOG.warn("fetch result timeout {}", backend.getBrpcAddress()); status.setStatus("query timeout"); return null; } @@ -265,18 +265,18 @@ private RowBatch getNextInternal(Status status, Backend backend) throws TExcepti } } catch (TimeoutException e) { futureResponse.cancel(true); - LOG.warn("fetch result timeout {}, addr {}", timeoutTs - currentTs, backend.getBrpcAdress()); + LOG.warn("fetch result timeout {}, addr {}", timeoutTs - currentTs, backend.getBrpcAddress()); status.setStatus("query timeout"); return null; } } } catch (RpcException e) { - LOG.warn("fetch result rpc exception {}, e {}", backend.getBrpcAdress(), e); + LOG.warn("fetch result rpc exception {}, e {}", backend.getBrpcAddress(), e); status.setRpcStatus(e.getMessage()); SimpleScheduler.addToBlacklist(backend.getId(), e.getMessage()); return null; } catch (ExecutionException e) { - LOG.warn("fetch result execution exception {}, addr {}", e, backend.getBrpcAdress()); + LOG.warn("fetch result execution exception {}, addr {}", e, backend.getBrpcAddress()); if (e.getMessage().contains("time out")) { // if timeout, we set error code to TIMEOUT, and it will not retry querying. status.setStatus(new Status(TStatusCode.TIMEOUT, e.getMessage())); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/QeService.java b/fe/fe-core/src/main/java/org/apache/doris/qe/QeService.java index 36eb4e8db59c97..f1e9a653454120 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/QeService.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/QeService.java @@ -18,6 +18,7 @@ package org.apache.doris.qe; import org.apache.doris.mysql.MysqlServer; +import org.apache.doris.service.arrowflight.FlightSqlService; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -33,13 +34,18 @@ public class QeService { // MySQL protocol service private MysqlServer mysqlServer; + private int arrowFlightSQLPort; + private FlightSqlService flightSqlService; + @Deprecated - public QeService(int port) { + public QeService(int port, int arrowFlightSQLPort) { this.port = port; + this.arrowFlightSQLPort = arrowFlightSQLPort; } - public QeService(int port, ConnectScheduler scheduler) { + public QeService(int port, int arrowFlightSQLPort, ConnectScheduler scheduler) { this.port = port; + this.arrowFlightSQLPort = arrowFlightSQLPort; this.mysqlServer = new MysqlServer(port, scheduler); } @@ -56,6 +62,14 @@ public void start() throws Exception { LOG.error("mysql server start failed"); System.exit(-1); } + if (arrowFlightSQLPort != -1) { + this.flightSqlService = new FlightSqlService(arrowFlightSQLPort); + if (!flightSqlService.start()) { + System.exit(-1); + } + } else { + LOG.info("No Arrow Flight SQL service that needs to be started."); + } LOG.info("QE service start."); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index ecedfa3aa289f9..6b30fc58e20f9e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -146,6 +146,7 @@ import org.apache.doris.rpc.BackendServiceProxy; import org.apache.doris.rpc.RpcException; import org.apache.doris.service.FrontendOptions; +import org.apache.doris.service.arrowflight.FlightStatementExecutor; import org.apache.doris.statistics.ResultRow; import org.apache.doris.statistics.util.InternalQueryBuffer; import org.apache.doris.system.Backend; @@ -2597,7 +2598,8 @@ public List executeInternalQuery() { planner = new NereidsPlanner(statementContext); planner.plan(parsedStmt, context.getSessionVariable().toThrift()); } catch (Exception e) { - LOG.warn("fall back to legacy planner, because: {}", e.getMessage(), e); + LOG.warn("Arrow Flight SQL fall back to legacy planner, because: {}", + e.getMessage(), e); parsedStmt = null; planner = null; context.getState().setNereids(false); @@ -2612,7 +2614,6 @@ public List executeInternalQuery() { LOG.warn("Failed to run internal SQL: {}", originStmt, e); throw new RuntimeException("Failed to execute internal SQL. " + Util.getRootCauseMessage(e), e); } - planner.getFragments(); RowBatch batch; coord = new Coordinator(context, analyzer, planner, context.getStatsErrorEstimator()); profile.addExecutionProfile(coord.getExecutionProfile()); @@ -2646,7 +2647,7 @@ public List executeInternalQuery() { } } catch (Exception e) { fetchResultSpan.recordException(e); - throw new RuntimeException("Failed to execute internal SQL. " + Util.getRootCauseMessage(e), e); + throw new RuntimeException("Failed to fetch internal SQL result. " + Util.getRootCauseMessage(e), e); } finally { fetchResultSpan.end(); } @@ -2657,6 +2658,64 @@ public List executeInternalQuery() { } } + public void executeArrowFlightQuery(FlightStatementExecutor flightStatementExecutor) { + LOG.debug("ARROW FLIGHT QUERY: " + originStmt.toString()); + try { + try { + if (ConnectContext.get() != null + && ConnectContext.get().getSessionVariable().isEnableNereidsPlanner()) { + try { + parseByNereids(); + Preconditions.checkState(parsedStmt instanceof LogicalPlanAdapter, + "Nereids only process LogicalPlanAdapter," + + " but parsedStmt is " + parsedStmt.getClass().getName()); + context.getState().setNereids(true); + context.getState().setIsQuery(true); + planner = new NereidsPlanner(statementContext); + planner.plan(parsedStmt, context.getSessionVariable().toThrift()); + } catch (Exception e) { + LOG.warn("fall back to legacy planner, because: {}", e.getMessage(), e); + parsedStmt = null; + context.getState().setNereids(false); + analyzer = new Analyzer(context.getEnv(), context); + analyze(context.getSessionVariable().toThrift()); + } + } else { + analyzer = new Analyzer(context.getEnv(), context); + analyze(context.getSessionVariable().toThrift()); + } + } catch (Exception e) { + throw new RuntimeException("Failed to execute Arrow Flight SQL. " + Util.getRootCauseMessage(e), e); + } + coord = new Coordinator(context, analyzer, planner, context.getStatsErrorEstimator()); + profile.addExecutionProfile(coord.getExecutionProfile()); + try { + QeProcessorImpl.INSTANCE.registerQuery(context.queryId(), + new QeProcessorImpl.QueryInfo(context, originStmt.originStmt, coord)); + } catch (UserException e) { + throw new RuntimeException("Failed to execute Arrow Flight SQL. " + Util.getRootCauseMessage(e), e); + } + + Span queryScheduleSpan = context.getTracer() + .spanBuilder("Arrow Flight SQL schedule").setParent(Context.current()).startSpan(); + try (Scope scope = queryScheduleSpan.makeCurrent()) { + coord.exec(); + } catch (Exception e) { + queryScheduleSpan.recordException(e); + LOG.warn("Failed to coord exec Arrow Flight SQL, because: {}", e.getMessage(), e); + throw new RuntimeException(e.getMessage() + Util.getRootCauseMessage(e), e); + } finally { + queryScheduleSpan.end(); + } + } finally { + QeProcessorImpl.INSTANCE.unregisterQuery(context.queryId()); // TODO for query profile + } + flightStatementExecutor.setFinstId(coord.getFinstId()); + flightStatementExecutor.setResultFlightServerAddr(coord.getResultFlightServerAddr()); + flightStatementExecutor.setResultInternalServiceAddr(coord.getResultInternalServiceAddr()); + flightStatementExecutor.setResultOutputExprs(coord.getResultOutputExprs()); + } + private List convertResultBatchToResultRows(TResultBatch batch) { List columns = parsedStmt.getColLabels(); List resultRows = new ArrayList<>(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java index e1f5a2c95b9095..3250d186889260 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java @@ -100,6 +100,11 @@ public InternalService.PFetchDataResult fetchDataSync(InternalService.PFetchData return blockingStub.fetchData(request); } + public Future fetchArrowFlightSchema( + InternalService.PFetchArrowFlightSchemaRequest request) { + return stub.fetchArrowFlightSchema(request); + } + public Future fetchTableStructureAsync( InternalService.PFetchTableSchemaRequest request) { return stub.fetchTableSchema(request); diff --git a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java index b30b8e0c5a2f86..55881b4cf91418 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java @@ -259,6 +259,18 @@ public InternalService.PFetchDataResult fetchDataSync( } } + public Future fetchArrowFlightSchema( + TNetworkAddress address, InternalService.PFetchArrowFlightSchemaRequest request) throws RpcException { + try { + final BackendServiceClient client = getProxy(address); + return client.fetchArrowFlightSchema(request); + } catch (Throwable e) { + LOG.warn("fetch arrow flight schema catch a exception, address={}:{}", + address.getHostname(), address.getPort(), e); + throw new RpcException(address.hostname, e.getMessage()); + } + } + public Future fetchTableStructureAsync( TNetworkAddress address, InternalService.PFetchTableSchemaRequest request) throws RpcException { try { 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 b0796a372de69e..4238e012c5ef2c 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 @@ -2226,6 +2226,7 @@ public TFrontendPingFrontendResult ping(TFrontendPingFrontendRequest request) th result.setReplayedJournalId(replayedJournalId); result.setQueryPort(Config.query_port); result.setRpcPort(Config.rpc_port); + result.setArrowFlightSqlPort(Config.arrow_flight_sql_port); result.setVersion(Version.DORIS_BUILD_VERSION + "-" + Version.DORIS_BUILD_SHORT_HASH); result.setLastStartupTime(exeEnv.getStartupTime()); result.setProcessUUID(exeEnv.getProcessUUID()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightSqlService.java b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightSqlService.java new file mode 100644 index 00000000000000..e0ec4bf10c08ee --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightSqlService.java @@ -0,0 +1,67 @@ +// 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.service.arrowflight; + +import org.apache.arrow.flight.FlightServer; +import org.apache.arrow.flight.Location; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.IOException; + +/** + * flight sql protocol implementation based on nio. + */ +public class FlightSqlService { + private static final Logger LOG = LogManager.getLogger(FlightSqlService.class); + private final FlightServer flightServer; + private volatile boolean running; + + public FlightSqlService(int port) { + BufferAllocator allocator = new RootAllocator(); + Location location = Location.forGrpcInsecure("0.0.0.0", port); + FlightSqlServiceImpl producer = new FlightSqlServiceImpl(location); + flightServer = FlightServer.builder(allocator, location, producer).build(); + } + + // start Arrow Flight SQL service, return true if success, otherwise false + public boolean start() { + try { + flightServer.start(); + running = true; + LOG.info("Arrow Flight SQL service is started."); + } catch (IOException e) { + LOG.error("Start Arrow Flight SQL service failed.", e); + return false; + } + return true; + } + + public void stop() { + if (running) { + running = false; + try { + flightServer.close(); + } catch (InterruptedException e) { + LOG.warn("close Arrow Flight SQL server failed.", e); + } + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightSqlServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightSqlServiceImpl.java new file mode 100644 index 00000000000000..38e275b1d5dd0a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightSqlServiceImpl.java @@ -0,0 +1,326 @@ +// 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/apache/arrow/blob/main/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/FlightSqlExample.java +// and modified by Doris + +package org.apache.doris.service.arrowflight; + +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.common.util.Util; + +import com.google.protobuf.Any; +import com.google.protobuf.ByteString; +import com.google.protobuf.Message; +import org.apache.arrow.flight.CallStatus; +import org.apache.arrow.flight.Criteria; +import org.apache.arrow.flight.FlightDescriptor; +import org.apache.arrow.flight.FlightEndpoint; +import org.apache.arrow.flight.FlightInfo; +import org.apache.arrow.flight.FlightStream; +import org.apache.arrow.flight.Location; +import org.apache.arrow.flight.PutResult; +import org.apache.arrow.flight.Result; +import org.apache.arrow.flight.SchemaResult; +import org.apache.arrow.flight.Ticket; +import org.apache.arrow.flight.sql.FlightSqlProducer; +import org.apache.arrow.flight.sql.SqlInfoBuilder; +import org.apache.arrow.flight.sql.impl.FlightSql.ActionClosePreparedStatementRequest; +import org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementRequest; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetCatalogs; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetCrossReference; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetDbSchemas; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetExportedKeys; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetImportedKeys; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetPrimaryKeys; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetSqlInfo; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetTableTypes; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetTables; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetXdbcTypeInfo; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementQuery; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementUpdate; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandStatementQuery; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandStatementUpdate; +import org.apache.arrow.flight.sql.impl.FlightSql.SqlSupportedCaseSensitivity; +import org.apache.arrow.flight.sql.impl.FlightSql.TicketStatementQuery; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.util.AutoCloseables; +import org.apache.arrow.vector.types.pojo.Schema; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Collections; +import java.util.List; + +public class FlightSqlServiceImpl implements FlightSqlProducer, AutoCloseable { + private static final Logger LOG = LogManager.getLogger(FlightSqlServiceImpl.class); + private final Location location; + private final BufferAllocator rootAllocator = new RootAllocator(); + private final SqlInfoBuilder sqlInfoBuilder; + + public FlightSqlServiceImpl(final Location location) { + this.location = location; + sqlInfoBuilder = new SqlInfoBuilder(); + sqlInfoBuilder.withFlightSqlServerName("DorisFE") + .withFlightSqlServerVersion("1.0") + .withFlightSqlServerArrowVersion("13.0") + .withFlightSqlServerReadOnly(false) + .withSqlIdentifierQuoteChar("`") + .withSqlDdlCatalog(true) + .withSqlDdlSchema(false) + .withSqlDdlTable(false) + .withSqlIdentifierCase(SqlSupportedCaseSensitivity.SQL_CASE_SENSITIVITY_CASE_INSENSITIVE) + .withSqlQuotedIdentifierCase(SqlSupportedCaseSensitivity.SQL_CASE_SENSITIVITY_CASE_INSENSITIVE); + } + + @Override + public void getStreamPreparedStatement(final CommandPreparedStatementQuery command, final CallContext context, + final ServerStreamListener listener) { + throw CallStatus.UNIMPLEMENTED.withDescription("getStreamPreparedStatement unimplemented").toRuntimeException(); + } + + @Override + public void closePreparedStatement(final ActionClosePreparedStatementRequest request, final CallContext context, + final StreamListener listener) { + throw CallStatus.UNIMPLEMENTED.withDescription("closePreparedStatement unimplemented").toRuntimeException(); + } + + @Override + public FlightInfo getFlightInfoStatement(final CommandStatementQuery request, final CallContext context, + final FlightDescriptor descriptor) { + try { + final String query = request.getQuery(); + final FlightStatementExecutor flightStatementExecutor = new FlightStatementExecutor(query); + + flightStatementExecutor.executeQuery(); + + TicketStatementQuery ticketStatement = TicketStatementQuery.newBuilder() + .setStatementHandle(ByteString.copyFromUtf8( + DebugUtil.printId(flightStatementExecutor.getFinstId()) + ":" + query)).build(); + final Ticket ticket = new Ticket(Any.pack(ticketStatement).toByteArray()); + // TODO Support multiple endpoints. + Location location = Location.forGrpcInsecure(flightStatementExecutor.getResultFlightServerAddr().hostname, + flightStatementExecutor.getResultFlightServerAddr().port); + List endpoints = Collections.singletonList(new FlightEndpoint(ticket, location)); + + Schema schema; + schema = flightStatementExecutor.fetchArrowFlightSchema(5000); + if (schema == null) { + throw CallStatus.INTERNAL.withDescription("fetch arrow flight schema is null").toRuntimeException(); + } + return new FlightInfo(schema, descriptor, endpoints, -1, -1); + } catch (Exception e) { + LOG.warn("get flight info statement failed, " + e.getMessage(), e); + throw CallStatus.INTERNAL.withDescription(Util.getRootCauseMessage(e)).withCause(e).toRuntimeException(); + } + } + + @Override + public FlightInfo getFlightInfoPreparedStatement(final CommandPreparedStatementQuery command, + final CallContext context, + final FlightDescriptor descriptor) { + throw CallStatus.UNIMPLEMENTED.withDescription("getFlightInfoPreparedStatement unimplemented") + .toRuntimeException(); + } + + @Override + public SchemaResult getSchemaStatement(final CommandStatementQuery command, final CallContext context, + final FlightDescriptor descriptor) { + throw CallStatus.UNIMPLEMENTED.withDescription("getSchemaStatement unimplemented").toRuntimeException(); + } + + @Override + public void close() throws Exception { + AutoCloseables.close(rootAllocator); + } + + @Override + public void listFlights(CallContext context, Criteria criteria, StreamListener listener) { + throw CallStatus.UNIMPLEMENTED.withDescription("listFlights unimplemented").toRuntimeException(); + } + + @Override + public void createPreparedStatement(final ActionCreatePreparedStatementRequest request, final CallContext context, + final StreamListener listener) { + throw CallStatus.UNIMPLEMENTED.withDescription("createPreparedStatement unimplemented").toRuntimeException(); + } + + @Override + public void doExchange(CallContext context, FlightStream reader, ServerStreamListener writer) { + throw CallStatus.UNIMPLEMENTED.withDescription("doExchange unimplemented").toRuntimeException(); + } + + @Override + public Runnable acceptPutStatement(CommandStatementUpdate command, + CallContext context, FlightStream flightStream, + StreamListener ackStream) { + throw CallStatus.UNIMPLEMENTED.withDescription("acceptPutStatement unimplemented").toRuntimeException(); + } + + @Override + public Runnable acceptPutPreparedStatementUpdate(CommandPreparedStatementUpdate command, CallContext context, + FlightStream flightStream, StreamListener ackStream) { + throw CallStatus.UNIMPLEMENTED.withDescription("acceptPutPreparedStatementUpdate unimplemented") + .toRuntimeException(); + } + + @Override + public Runnable acceptPutPreparedStatementQuery(CommandPreparedStatementQuery command, CallContext context, + FlightStream flightStream, StreamListener ackStream) { + throw CallStatus.UNIMPLEMENTED.withDescription("acceptPutPreparedStatementQuery unimplemented") + .toRuntimeException(); + } + + @Override + public FlightInfo getFlightInfoSqlInfo(final CommandGetSqlInfo request, final CallContext context, + final FlightDescriptor descriptor) { + return getFlightInfoForSchema(request, descriptor, Schemas.GET_SQL_INFO_SCHEMA); + } + + @Override + public void getStreamSqlInfo(final CommandGetSqlInfo command, final CallContext context, + final ServerStreamListener listener) { + this.sqlInfoBuilder.send(command.getInfoList(), listener); + } + + @Override + public FlightInfo getFlightInfoTypeInfo(CommandGetXdbcTypeInfo request, CallContext context, + FlightDescriptor descriptor) { + return getFlightInfoForSchema(request, descriptor, Schemas.GET_TYPE_INFO_SCHEMA); + } + + @Override + public void getStreamTypeInfo(CommandGetXdbcTypeInfo request, CallContext context, + ServerStreamListener listener) { + throw CallStatus.UNIMPLEMENTED.withDescription("getStreamTypeInfo unimplemented").toRuntimeException(); + } + + @Override + public FlightInfo getFlightInfoCatalogs(final CommandGetCatalogs request, final CallContext context, + final FlightDescriptor descriptor) { + return getFlightInfoForSchema(request, descriptor, Schemas.GET_CATALOGS_SCHEMA); + } + + @Override + public void getStreamCatalogs(final CallContext context, final ServerStreamListener listener) { + throw CallStatus.UNIMPLEMENTED.withDescription("getStreamCatalogs unimplemented").toRuntimeException(); + } + + @Override + public FlightInfo getFlightInfoSchemas(final CommandGetDbSchemas request, final CallContext context, + final FlightDescriptor descriptor) { + return getFlightInfoForSchema(request, descriptor, Schemas.GET_SCHEMAS_SCHEMA); + } + + @Override + public void getStreamSchemas(final CommandGetDbSchemas command, final CallContext context, + final ServerStreamListener listener) { + throw CallStatus.UNIMPLEMENTED.withDescription("getStreamSchemas unimplemented").toRuntimeException(); + } + + @Override + public FlightInfo getFlightInfoTables(final CommandGetTables request, final CallContext context, + final FlightDescriptor descriptor) { + Schema schemaToUse = Schemas.GET_TABLES_SCHEMA; + if (!request.getIncludeSchema()) { + schemaToUse = Schemas.GET_TABLES_SCHEMA_NO_SCHEMA; + } + return getFlightInfoForSchema(request, descriptor, schemaToUse); + } + + @Override + public void getStreamTables(final CommandGetTables command, final CallContext context, + final ServerStreamListener listener) { + throw CallStatus.UNIMPLEMENTED.withDescription("getStreamTables unimplemented").toRuntimeException(); + } + + @Override + public FlightInfo getFlightInfoTableTypes(final CommandGetTableTypes request, final CallContext context, + final FlightDescriptor descriptor) { + return getFlightInfoForSchema(request, descriptor, Schemas.GET_TABLE_TYPES_SCHEMA); + } + + @Override + public void getStreamTableTypes(final CallContext context, final ServerStreamListener listener) { + throw CallStatus.UNIMPLEMENTED.withDescription("getStreamTableTypes unimplemented").toRuntimeException(); + } + + @Override + public FlightInfo getFlightInfoPrimaryKeys(final CommandGetPrimaryKeys request, final CallContext context, + final FlightDescriptor descriptor) { + return getFlightInfoForSchema(request, descriptor, Schemas.GET_PRIMARY_KEYS_SCHEMA); + } + + @Override + public void getStreamPrimaryKeys(final CommandGetPrimaryKeys command, final CallContext context, + final ServerStreamListener listener) { + + throw CallStatus.UNIMPLEMENTED.withDescription("getStreamPrimaryKeys unimplemented").toRuntimeException(); + } + + @Override + public FlightInfo getFlightInfoExportedKeys(final CommandGetExportedKeys request, final CallContext context, + final FlightDescriptor descriptor) { + return getFlightInfoForSchema(request, descriptor, Schemas.GET_EXPORTED_KEYS_SCHEMA); + } + + @Override + public void getStreamExportedKeys(final CommandGetExportedKeys command, final CallContext context, + final ServerStreamListener listener) { + throw CallStatus.UNIMPLEMENTED.withDescription("getStreamExportedKeys unimplemented").toRuntimeException(); + } + + @Override + public FlightInfo getFlightInfoImportedKeys(final CommandGetImportedKeys request, final CallContext context, + final FlightDescriptor descriptor) { + return getFlightInfoForSchema(request, descriptor, Schemas.GET_IMPORTED_KEYS_SCHEMA); + } + + @Override + public void getStreamImportedKeys(final CommandGetImportedKeys command, final CallContext context, + final ServerStreamListener listener) { + throw CallStatus.UNIMPLEMENTED.withDescription("getStreamImportedKeys unimplemented").toRuntimeException(); + } + + @Override + public FlightInfo getFlightInfoCrossReference(CommandGetCrossReference request, CallContext context, + FlightDescriptor descriptor) { + return getFlightInfoForSchema(request, descriptor, Schemas.GET_CROSS_REFERENCE_SCHEMA); + } + + @Override + public void getStreamCrossReference(CommandGetCrossReference command, CallContext context, + ServerStreamListener listener) { + throw CallStatus.UNIMPLEMENTED.withDescription("getStreamCrossReference unimplemented").toRuntimeException(); + } + + @Override + public void getStreamStatement(final TicketStatementQuery ticketStatementQuery, final CallContext context, + final ServerStreamListener listener) { + throw CallStatus.UNIMPLEMENTED.withDescription("getStreamStatement unimplemented").toRuntimeException(); + } + + private FlightInfo getFlightInfoForSchema(final T request, final FlightDescriptor descriptor, + final Schema schema) { + final Ticket ticket = new Ticket(Any.pack(request).toByteArray()); + // TODO Support multiple endpoints. + final List endpoints = Collections.singletonList(new FlightEndpoint(ticket, location)); + + return new FlightInfo(schema, descriptor, endpoints, -1, -1); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightStatementExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightStatementExecutor.java new file mode 100644 index 00000000000000..ced03350de4e05 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightStatementExecutor.java @@ -0,0 +1,224 @@ +// 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/apache/arrow/blob/main/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/StatementContext.java +// and modified by Doris + +package org.apache.doris.service.arrowflight; + +import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.Status; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.proto.InternalService; +import org.apache.doris.proto.Types; +import org.apache.doris.qe.AutoCloseConnectContext; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.SessionVariable; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.rpc.BackendServiceProxy; +import org.apache.doris.rpc.RpcException; +import org.apache.doris.system.SystemInfoService; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TResultSinkType; +import org.apache.doris.thrift.TStatusCode; +import org.apache.doris.thrift.TUniqueId; + +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.ipc.ArrowStreamReader; +import org.apache.arrow.vector.types.pojo.Schema; + +import java.io.ByteArrayInputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public final class FlightStatementExecutor { + private AutoCloseConnectContext acConnectContext; + private final String query; + private TUniqueId queryId; + private TUniqueId finstId; + private TNetworkAddress resultFlightServerAddr; + private TNetworkAddress resultInternalServiceAddr; + private ArrayList resultOutputExprs; + + public FlightStatementExecutor(final String query) { + this.query = query; + acConnectContext = buildConnectContext(); + } + + public void setQueryId(TUniqueId queryId) { + this.queryId = queryId; + } + + public void setFinstId(TUniqueId finstId) { + this.finstId = finstId; + } + + public void setResultFlightServerAddr(TNetworkAddress resultFlightServerAddr) { + this.resultFlightServerAddr = resultFlightServerAddr; + } + + public void setResultInternalServiceAddr(TNetworkAddress resultInternalServiceAddr) { + this.resultInternalServiceAddr = resultInternalServiceAddr; + } + + public void setResultOutputExprs(ArrayList resultOutputExprs) { + this.resultOutputExprs = resultOutputExprs; + } + + public String getQuery() { + return query; + } + + public TUniqueId getQueryId() { + return queryId; + } + + public TUniqueId getFinstId() { + return finstId; + } + + public TNetworkAddress getResultFlightServerAddr() { + return resultFlightServerAddr; + } + + public TNetworkAddress getResultInternalServiceAddr() { + return resultInternalServiceAddr; + } + + public ArrayList getResultOutputExprs() { + return resultOutputExprs; + } + + @Override + public boolean equals(final Object other) { + if (!(other instanceof FlightStatementExecutor)) { + return false; + } + return this == other; + } + + @Override + public int hashCode() { + return Objects.hash(this); + } + + public static AutoCloseConnectContext buildConnectContext() { + ConnectContext connectContext = new ConnectContext(); + SessionVariable sessionVariable = connectContext.getSessionVariable(); + sessionVariable.internalSession = true; + sessionVariable.setEnablePipelineEngine(false); // TODO + sessionVariable.setEnablePipelineXEngine(false); // TODO + connectContext.setEnv(Env.getCurrentEnv()); + connectContext.setQualifiedUser(UserIdentity.ROOT.getQualifiedUser()); // TODO + connectContext.setCurrentUserIdentity(UserIdentity.ROOT); // TODO + connectContext.setStartTime(); + connectContext.setCluster(SystemInfoService.DEFAULT_CLUSTER); + connectContext.setResultSinkType(TResultSinkType.ARROW_FLIGHT_PROTOCAL); + return new AutoCloseConnectContext(connectContext); + } + + public void executeQuery() { + try { + UUID uuid = UUID.randomUUID(); + TUniqueId queryId = new TUniqueId(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits()); + setQueryId(queryId); + acConnectContext.connectContext.setQueryId(queryId); + StmtExecutor stmtExecutor = new StmtExecutor(acConnectContext.connectContext, getQuery()); + acConnectContext.connectContext.setExecutor(stmtExecutor); + stmtExecutor.executeArrowFlightQuery(this); + } catch (Exception e) { + throw new RuntimeException("Failed to coord exec", e); + } + } + + public Schema fetchArrowFlightSchema(int timeoutMs) { + TNetworkAddress address = getResultInternalServiceAddr(); + TUniqueId tid = getFinstId(); + ArrayList resultOutputExprs = getResultOutputExprs(); + Types.PUniqueId finstId = Types.PUniqueId.newBuilder().setHi(tid.hi).setLo(tid.lo).build(); + try { + InternalService.PFetchArrowFlightSchemaRequest request = + InternalService.PFetchArrowFlightSchemaRequest.newBuilder() + .setFinstId(finstId) + .build(); + + Future future + = BackendServiceProxy.getInstance().fetchArrowFlightSchema(address, request); + InternalService.PFetchArrowFlightSchemaResult pResult; + pResult = future.get(timeoutMs, TimeUnit.MILLISECONDS); + if (pResult == null) { + throw new RuntimeException(String.format("fetch arrow flight schema timeout, finstId: %s", + DebugUtil.printId(tid))); + } + TStatusCode code = TStatusCode.findByValue(pResult.getStatus().getStatusCode()); + if (code != TStatusCode.OK) { + Status status = null; + status.setPstatus(pResult.getStatus()); + throw new RuntimeException(String.format("fetch arrow flight schema failed, finstId: %s, errmsg: %s", + DebugUtil.printId(tid), status)); + } + if (pResult.hasSchema() && pResult.getSchema().size() > 0) { + RootAllocator rootAllocator = new RootAllocator(Integer.MAX_VALUE); + ArrowStreamReader arrowStreamReader = new ArrowStreamReader( + new ByteArrayInputStream(pResult.getSchema().toByteArray()), + rootAllocator + ); + try { + VectorSchemaRoot root = arrowStreamReader.getVectorSchemaRoot(); + List fieldVectors = root.getFieldVectors(); + if (fieldVectors.size() != resultOutputExprs.size()) { + throw new RuntimeException(String.format( + "Schema size %s' is not equal to arrow field size %s, finstId: %s.", + fieldVectors.size(), resultOutputExprs.size(), DebugUtil.printId(tid))); + } + return root.getSchema(); + } catch (Exception e) { + throw new RuntimeException("Read Arrow Flight Schema failed.", e); + } + } else { + throw new RuntimeException(String.format("get empty arrow flight schema, finstId: %s", + DebugUtil.printId(tid))); + } + } catch (RpcException e) { + throw new RuntimeException(String.format( + "arrow flight schema fetch catch rpc exception, finstId: %s,backend: %s", + DebugUtil.printId(tid), address), e); + } catch (InterruptedException e) { + throw new RuntimeException(String.format( + "arrow flight schema future get interrupted exception, finstId: %s,backend: %s", + DebugUtil.printId(tid), address), e); + } catch (ExecutionException e) { + throw new RuntimeException(String.format( + "arrow flight schema future get execution exception, finstId: %s,backend: %s", + DebugUtil.printId(tid), address), e); + } catch (TimeoutException e) { + throw new RuntimeException(String.format( + "arrow flight schema fetch timeout, finstId: %s,backend: %s", + DebugUtil.printId(tid), address), e); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java b/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java index db470fb91ddcb6..fcb5e63e8383fd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java @@ -76,6 +76,8 @@ public class Backend implements Writable { private volatile int beRpcPort; // be rpc port @SerializedName("brpcPort") private volatile int brpcPort = -1; + @SerializedName("arrowFlightSqlPort") + private volatile int arrowFlightSqlPort = -1; @SerializedName("lastUpdateMs") private volatile long lastUpdateMs; @@ -204,6 +206,10 @@ public int getBrpcPort() { return brpcPort; } + public int getArrowFlightSqlPort() { + return arrowFlightSqlPort; + } + public String getHeartbeatErrMsg() { return heartbeatErrMsg; } @@ -289,6 +295,10 @@ public void setBrpcPort(int brpcPort) { this.brpcPort = brpcPort; } + public void setArrowFlightSqlPort(int arrowFlightSqlPort) { + this.arrowFlightSqlPort = arrowFlightSqlPort; + } + public void setCpuCores(int cpuCores) { this.cpuCores = cpuCores; } @@ -670,6 +680,11 @@ public boolean handleHbResponse(BackendHbResponse hbResponse, boolean isReplay) this.brpcPort = hbResponse.getBrpcPort(); } + if (this.arrowFlightSqlPort != hbResponse.getArrowFlightSqlPort() && !FeConstants.runningUnitTest) { + isChanged = true; + this.arrowFlightSqlPort = hbResponse.getArrowFlightSqlPort(); + } + if (this.isShutDown.get() != hbResponse.isShutDown()) { isChanged = true; LOG.info("{} shutdown state is changed", this.toString()); @@ -796,10 +811,14 @@ public Map getTagMap() { return tagMap; } - public TNetworkAddress getBrpcAdress() { + public TNetworkAddress getBrpcAddress() { return new TNetworkAddress(getHost(), getBrpcPort()); } + public TNetworkAddress getArrowFlightAddress() { + return new TNetworkAddress(getHost(), getArrowFlightSqlPort()); + } + public String getTagMapString() { return "{" + new PrintableMap<>(tagMap, ":", true, false).toString() + "}"; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/BackendHbResponse.java b/fe/fe-core/src/main/java/org/apache/doris/system/BackendHbResponse.java index 18c5b94568b9c0..a91dd12b049b39 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/BackendHbResponse.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/BackendHbResponse.java @@ -37,6 +37,8 @@ public class BackendHbResponse extends HeartbeatResponse implements Writable { private int httpPort; @SerializedName(value = "brpcPort") private int brpcPort; + @SerializedName(value = "arrowFlightSqlPort") + private int arrowFlightSqlPort; @SerializedName(value = "nodeRole") private String nodeRole = Tag.VALUE_MIX; @@ -54,7 +56,7 @@ public BackendHbResponse() { } public BackendHbResponse(long beId, int bePort, int httpPort, int brpcPort, long hbTime, long beStartTime, - String version, String nodeRole, boolean isShutDown) { + String version, String nodeRole, boolean isShutDown, int arrowFlightSqlPort) { super(HeartbeatResponse.Type.BACKEND); this.beId = beId; this.status = HbStatus.OK; @@ -66,6 +68,7 @@ public BackendHbResponse(long beId, int bePort, int httpPort, int brpcPort, long this.version = version; this.nodeRole = nodeRole; this.isShutDown = isShutDown; + this.arrowFlightSqlPort = arrowFlightSqlPort; } public BackendHbResponse(long beId, String errMsg) { @@ -99,6 +102,10 @@ public int getBrpcPort() { return brpcPort; } + public int getArrowFlightSqlPort() { + return arrowFlightSqlPort; + } + public long getBeStartTime() { return beStartTime; } @@ -122,6 +129,7 @@ protected void readFields(DataInput in) throws IOException { bePort = in.readInt(); httpPort = in.readInt(); brpcPort = in.readInt(); + arrowFlightSqlPort = in.readInt(); } @Override @@ -133,6 +141,7 @@ public String toString() { sb.append(", bePort: ").append(bePort); sb.append(", httpPort: ").append(httpPort); sb.append(", brpcPort: ").append(brpcPort); + sb.append(", arrowFlightSqlPort: ").append(arrowFlightSqlPort); return sb.toString(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/Frontend.java b/fe/fe-core/src/main/java/org/apache/doris/system/Frontend.java index 95937b9d42013a..51e236974602b7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/Frontend.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/Frontend.java @@ -51,6 +51,7 @@ public class Frontend implements Writable { private int queryPort; private int rpcPort; + private int arrowFlightSqlPort; private long replayedJournalId; private long lastStartupTime; @@ -100,6 +101,10 @@ public int getRpcPort() { return rpcPort; } + public int getArrowFlightSqlPort() { + return arrowFlightSqlPort; + } + public boolean isAlive() { return isAlive; } @@ -153,6 +158,7 @@ public boolean handleHbResponse(FrontendHbResponse hbResponse, boolean isReplay) version = hbResponse.getVersion(); queryPort = hbResponse.getQueryPort(); rpcPort = hbResponse.getRpcPort(); + arrowFlightSqlPort = hbResponse.getArrowFlightSqlPort(); replayedJournalId = hbResponse.getReplayedJournalId(); lastUpdateTime = hbResponse.getHbTime(); heartbeatErrMsg = ""; diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/FrontendHbResponse.java b/fe/fe-core/src/main/java/org/apache/doris/system/FrontendHbResponse.java index f7d7e90624d022..c9afcef49b37f3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/FrontendHbResponse.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/FrontendHbResponse.java @@ -39,6 +39,8 @@ public class FrontendHbResponse extends HeartbeatResponse implements Writable { private int queryPort; @SerializedName(value = "rpcPort") private int rpcPort; + @SerializedName(value = "arrowFlightSqlPort") + private int arrowFlightSqlPort; @SerializedName(value = "replayedJournalId") private long replayedJournalId; private String version; @@ -50,7 +52,7 @@ public FrontendHbResponse() { super(HeartbeatResponse.Type.FRONTEND); } - public FrontendHbResponse(String name, int queryPort, int rpcPort, + public FrontendHbResponse(String name, int queryPort, int rpcPort, int arrowFlightSqlPort, long replayedJournalId, long hbTime, String version, long feStartTime, List diskInfos, long processUUID) { @@ -59,6 +61,7 @@ public FrontendHbResponse(String name, int queryPort, int rpcPort, this.name = name; this.queryPort = queryPort; this.rpcPort = rpcPort; + this.arrowFlightSqlPort = arrowFlightSqlPort; this.replayedJournalId = replayedJournalId; this.hbTime = hbTime; this.version = version; @@ -87,6 +90,10 @@ public int getRpcPort() { return rpcPort; } + public int getArrowFlightSqlPort() { + return arrowFlightSqlPort; + } + public long getReplayedJournalId() { return replayedJournalId; } @@ -113,6 +120,7 @@ public void readFields(DataInput in) throws IOException { name = Text.readString(in); queryPort = in.readInt(); rpcPort = in.readInt(); + arrowFlightSqlPort = in.readInt(); replayedJournalId = in.readLong(); } @@ -124,6 +132,7 @@ public String toString() { sb.append(", version: ").append(version); sb.append(", queryPort: ").append(queryPort); sb.append(", rpcPort: ").append(rpcPort); + sb.append(", arrowFlightSqlPort: ").append(arrowFlightSqlPort); sb.append(", replayedJournalId: ").append(replayedJournalId); sb.append(", festartTime: ").append(processUUID); return sb.toString(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatMgr.java b/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatMgr.java index 7dc1275afe79a1..a285d529a268f8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatMgr.java @@ -238,6 +238,7 @@ public HeartbeatResponse call() { backendInfo.setHttpPort(2); backendInfo.setBeRpcPort(3); backendInfo.setBrpcPort(4); + backendInfo.setArrowFlightSqlPort(8); backendInfo.setVersion("test-1234"); result = new THeartbeatResult(); result.setStatus(new TStatus(TStatusCode.OK)); @@ -253,6 +254,10 @@ public HeartbeatResponse call() { if (tBackendInfo.isSetBrpcPort()) { brpcPort = tBackendInfo.getBrpcPort(); } + int arrowFlightSqlPort = -1; + if (tBackendInfo.isSetArrowFlightSqlPort()) { + arrowFlightSqlPort = tBackendInfo.getArrowFlightSqlPort(); + } String version = ""; if (tBackendInfo.isSetVersion()) { version = tBackendInfo.getVersion(); @@ -267,7 +272,7 @@ public HeartbeatResponse call() { isShutDown = tBackendInfo.isIsShutdown(); } return new BackendHbResponse(backendId, bePort, httpPort, brpcPort, - System.currentTimeMillis(), beStartTime, version, nodeRole, isShutDown); + System.currentTimeMillis(), beStartTime, version, nodeRole, isShutDown, arrowFlightSqlPort); } else { return new BackendHbResponse(backendId, backend.getHost(), result.getStatus().getErrorMsgs().isEmpty() @@ -308,7 +313,8 @@ public HeartbeatResponse call() { // heartbeat to self if (Env.getCurrentEnv().isReady()) { return new FrontendHbResponse(fe.getNodeName(), Config.query_port, Config.rpc_port, - Env.getCurrentEnv().getMaxJournalId(), System.currentTimeMillis(), + Config.arrow_flight_sql_port, Env.getCurrentEnv().getMaxJournalId(), + System.currentTimeMillis(), Version.DORIS_BUILD_VERSION + "-" + Version.DORIS_BUILD_SHORT_HASH, ExecuteEnv.getInstance().getStartupTime(), ExecuteEnv.getInstance().getDiskInfos(), ExecuteEnv.getInstance().getProcessUUID()); @@ -331,7 +337,7 @@ private HeartbeatResponse getHeartbeatResponse() { ok = true; if (result.getStatus() == TFrontendPingFrontendStatusCode.OK) { return new FrontendHbResponse(fe.getNodeName(), result.getQueryPort(), - result.getRpcPort(), result.getReplayedJournalId(), + result.getRpcPort(), result.getArrowFlightSqlPort(), result.getReplayedJournalId(), System.currentTimeMillis(), result.getVersion(), result.getLastStartupTime(), FeDiskInfo.fromThrifts(result.getDiskInfos()), result.getProcessUUID()); } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java index 9cf9f5ad27988e..69af28dffda8f0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java @@ -788,6 +788,7 @@ public void updateBackendState(Backend be) { memoryBe.setHttpPort(be.getHttpPort()); memoryBe.setBeRpcPort(be.getBeRpcPort()); memoryBe.setBrpcPort(be.getBrpcPort()); + memoryBe.setArrowFlightSqlPort(be.getArrowFlightSqlPort()); memoryBe.setLastUpdateMs(be.getLastUpdateMs()); memoryBe.setLastStartTime(be.getLastStartTime()); memoryBe.setDisks(be.getDisks()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/BackendsTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/BackendsTableValuedFunction.java index 3141c240fb31c4..4e6fe27398cf38 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/BackendsTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/BackendsTableValuedFunction.java @@ -45,6 +45,7 @@ public class BackendsTableValuedFunction extends MetadataTableValuedFunction { new Column("BePort", ScalarType.createType(PrimitiveType.INT)), new Column("HttpPort", ScalarType.createType(PrimitiveType.INT)), new Column("BrpcPort", ScalarType.createType(PrimitiveType.INT)), + new Column("ArrowFlightSqlPort", ScalarType.createType(PrimitiveType.INT)), new Column("LastStartTime", ScalarType.createStringType()), new Column("LastHeartbeat", ScalarType.createStringType()), new Column("Alive", ScalarType.createType(PrimitiveType.BOOLEAN)), diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/FrontendsTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/FrontendsTableValuedFunction.java index 92109d05a9a71c..d23c9dfd6fdc41 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/FrontendsTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/FrontendsTableValuedFunction.java @@ -44,6 +44,7 @@ public class FrontendsTableValuedFunction extends MetadataTableValuedFunction { new Column("HttpPort", ScalarType.createStringType()), new Column("QueryPort", ScalarType.createStringType()), new Column("RpcPort", ScalarType.createStringType()), + new Column("ArrowFlightSqlPort", ScalarType.createStringType()), new Column("Role", ScalarType.createStringType()), new Column("IsMaster", ScalarType.createStringType()), new Column("ClusterId", ScalarType.createStringType()), diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/LocalTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/LocalTableValuedFunction.java index 129c3f930c75bc..bf78faec95eb13 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/LocalTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/LocalTableValuedFunction.java @@ -92,7 +92,7 @@ private void getFileListFromBackend() throws AnalysisException { } BackendServiceProxy proxy = BackendServiceProxy.getInstance(); - TNetworkAddress address = be.getBrpcAdress(); + TNetworkAddress address = be.getBrpcAddress(); InternalService.PGlobRequest.Builder requestBuilder = InternalService.PGlobRequest.newBuilder(); requestBuilder.setPattern(filePath); try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java index a2d2599599ab7b..7b9d3f892eacc1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java @@ -188,6 +188,7 @@ private static TFetchSchemaTableDataResult backendsMetadataResult(TMetadataTable trow.addToColumnValue(new TCell().setIntVal(backend.getBePort())); trow.addToColumnValue(new TCell().setIntVal(backend.getHttpPort())); trow.addToColumnValue(new TCell().setIntVal(backend.getBrpcPort())); + trow.addToColumnValue(new TCell().setIntVal(backend.getArrowFlightSqlPort())); } trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(backend.getLastStartTime()))); trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(backend.getLastUpdateMs()))); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/UnsupportedTypeTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/UnsupportedTypeTest.java index 20c26286c29e32..538bbb140b5ad2 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/UnsupportedTypeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/UnsupportedTypeTest.java @@ -64,25 +64,11 @@ public void testUnsupportedTypeThrowException() { "select array_range(10)", "select kmap from type_tb1", "select * from type_tb1", - "select jsonb_parse('{\"k1\":\"v31\",\"k2\":300}')", "select * from type_tb", }; - Class[] exceptions = { - null, - null, - null, - null, - null, - AnalysisException.class, - AnalysisException.class - }; - for (int i = 0; i < 5; ++i) { + for (int i = 0; i < sqls.length; ++i) { runPlanner(sqls[i]); } - for (int i = 5; i < sqls.length; ++i) { - int iCopy = i; - Assertions.assertThrows(exceptions[i], () -> runPlanner(sqls[iCopy])); - } } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/service/FrontendServiceImplTest.java b/fe/fe-core/src/test/java/org/apache/doris/service/FrontendServiceImplTest.java index 9a1264776af2ce..05cbb2ceb04f7e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/service/FrontendServiceImplTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/service/FrontendServiceImplTest.java @@ -153,7 +153,7 @@ public void testCreatePartitionList() throws Exception { TCreatePartitionResult partition = impl.createPartition(request); Assert.assertEquals(partition.getStatus().getStatusCode(), TStatusCode.OK); - Partition pbeijing = table.getPartition("pBEIJING"); - Assert.assertNotNull(pbeijing); + List pbs = (List) table.getAllPartitions(); + Assert.assertEquals(pbs.size(), 1); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/system/HeartbeatMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/system/HeartbeatMgrTest.java index d8c3877706b9e5..0ad1b0bb645d8f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/system/HeartbeatMgrTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/system/HeartbeatMgrTest.java @@ -95,6 +95,7 @@ public void invalidateObject(TNetworkAddress address, FrontendService.Client obj normalResult.setReplayedJournalId(191224); normalResult.setQueryPort(9131); normalResult.setRpcPort(9121); + normalResult.setArrowFlightSqlPort(9141); normalResult.setVersion("test"); TFrontendPingFrontendRequest badRequest = new TFrontendPingFrontendRequest(12345, "abcde"); @@ -123,6 +124,7 @@ public void invalidateObject(TNetworkAddress address, FrontendService.Client obj Assert.assertEquals(191224, hbResponse.getReplayedJournalId()); Assert.assertEquals(9131, hbResponse.getQueryPort()); Assert.assertEquals(9121, hbResponse.getRpcPort()); + Assert.assertEquals(9141, hbResponse.getArrowFlightSqlPort()); Assert.assertEquals(HbStatus.OK, hbResponse.getStatus()); Assert.assertEquals("test", hbResponse.getVersion()); @@ -135,6 +137,7 @@ public void invalidateObject(TNetworkAddress address, FrontendService.Client obj Assert.assertEquals(0, hbResponse.getReplayedJournalId()); Assert.assertEquals(0, hbResponse.getQueryPort()); Assert.assertEquals(0, hbResponse.getRpcPort()); + Assert.assertEquals(0, hbResponse.getArrowFlightSqlPort()); Assert.assertEquals(HbStatus.BAD, hbResponse.getStatus()); Assert.assertEquals("not ready", hbResponse.getMsg()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/system/SystemInfoServiceTest.java b/fe/fe-core/src/test/java/org/apache/doris/system/SystemInfoServiceTest.java index 3c50bd47c8de13..7c5556e8cf2d49 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/system/SystemInfoServiceTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/system/SystemInfoServiceTest.java @@ -97,7 +97,7 @@ public void testBackendHbResponseSerialization() throws IOException { System.out.println(Env.getCurrentEnvJournalVersion()); BackendHbResponse writeResponse = new BackendHbResponse(1L, 1234, 1234, 1234, 1234, 1234, "test", - Tag.VALUE_COMPUTATION, false); + Tag.VALUE_COMPUTATION, false, 1234); // Write objects to file File file1 = new File("./BackendHbResponseSerialization"); diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/AnotherDemoTest.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/AnotherDemoTest.java index 6f3cf22d4444d4..98a3893123e4fb 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/utframe/AnotherDemoTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/AnotherDemoTest.java @@ -52,12 +52,14 @@ public class AnotherDemoTest { private static int fe_http_port; private static int fe_rpc_port; private static int fe_query_port; + private static int fe_arrow_flight_sql_port; private static int fe_edit_log_port; private static int be_heartbeat_port; private static int be_thrift_port; private static int be_brpc_port; private static int be_http_port; + private static int be_arrow_flight_sql_port; // use a unique dir so that it won't be conflict with other unit test which // may also start a Mocked Frontend @@ -81,12 +83,14 @@ private static void getPorts() { fe_http_port = UtFrameUtils.findValidPort(); fe_rpc_port = UtFrameUtils.findValidPort(); fe_query_port = UtFrameUtils.findValidPort(); + fe_arrow_flight_sql_port = UtFrameUtils.findValidPort(); fe_edit_log_port = UtFrameUtils.findValidPort(); be_heartbeat_port = UtFrameUtils.findValidPort(); be_thrift_port = UtFrameUtils.findValidPort(); be_brpc_port = UtFrameUtils.findValidPort(); be_http_port = UtFrameUtils.findValidPort(); + be_arrow_flight_sql_port = UtFrameUtils.findValidPort(); } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/DemoMultiBackendsTest.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/DemoMultiBackendsTest.java index 83c75f052b1211..10f94adeb8ca9d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/utframe/DemoMultiBackendsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/DemoMultiBackendsTest.java @@ -198,7 +198,8 @@ public void testCreateDbAndTable() throws Exception { BackendsProcDir dir = new BackendsProcDir(Env.getCurrentSystemInfo()); ProcResult result = dir.fetchResult(); Assert.assertEquals(BackendsProcDir.TITLE_NAMES.size(), result.getColumnNames().size()); - Assert.assertEquals("{\"location\" : \"default\"}", result.getRows().get(0).get(18)); + Assert.assertEquals("{\"location\" : \"default\"}", + result.getRows().get(0).get(BackendsProcDir.TITLE_NAMES.size() - 6)); Assert.assertEquals( "{\"lastSuccessReportTabletsTime\":\"N/A\",\"lastStreamLoadTime\":-1,\"isQueryDisabled\":false,\"isLoadDisabled\":false}", result.getRows().get(0).get(BackendsProcDir.TITLE_NAMES.size() - 3)); diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/MockedBackendFactory.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/MockedBackendFactory.java index 3a09cae73bb434..55abcf9542acbe 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/utframe/MockedBackendFactory.java +++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/MockedBackendFactory.java @@ -86,10 +86,11 @@ public class MockedBackendFactory { public static final int BE_DEFAULT_THRIFT_PORT = 9060; public static final int BE_DEFAULT_BRPC_PORT = 8060; public static final int BE_DEFAULT_HTTP_PORT = 8040; + public static final int BE_DEFAULT_ARROW_FLIGHT_SQL_PORT = 8070; // create a mocked backend with customize parameters public static MockedBackend createBackend(String host, int heartbeatPort, int thriftPort, int brpcPort, - int httpPort, + int httpPort, int arrowFlightSqlPort, HeartbeatService.Iface hbService, BeThriftService beThriftService, PBackendServiceGrpc.PBackendServiceImplBase pBackendService) throws IOException { @@ -105,16 +106,20 @@ public static class DefaultHeartbeatServiceImpl implements HeartbeatService.Ifac private int beHttpPort; private int beBrpcPort; - public DefaultHeartbeatServiceImpl(int beThriftPort, int beHttpPort, int beBrpcPort) { + private int beArrowFlightSqlPort; + + public DefaultHeartbeatServiceImpl(int beThriftPort, int beHttpPort, int beBrpcPort, int beArrowFlightSqlPort) { this.beThriftPort = beThriftPort; this.beHttpPort = beHttpPort; this.beBrpcPort = beBrpcPort; + this.beArrowFlightSqlPort = beArrowFlightSqlPort; } @Override public THeartbeatResult heartbeat(TMasterInfo masterInfo) throws TException { TBackendInfo backendInfo = new TBackendInfo(beThriftPort, beHttpPort); backendInfo.setBrpcPort(beBrpcPort); + backendInfo.setArrowFlightSqlPort(beArrowFlightSqlPort); THeartbeatResult result = new THeartbeatResult(new TStatus(TStatusCode.OK), backendInfo); return result; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/MockedFrontend.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/MockedFrontend.java index 4ee38199713844..6382621d9051a5 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/utframe/MockedFrontend.java +++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/MockedFrontend.java @@ -79,6 +79,7 @@ public class MockedFrontend { MIN_FE_CONF.put("http_port", "8030"); MIN_FE_CONF.put("rpc_port", "9020"); MIN_FE_CONF.put("query_port", "9030"); + MIN_FE_CONF.put("arrow_flight_sql_port", "9040"); MIN_FE_CONF.put("edit_log_port", "9010"); MIN_FE_CONF.put("priority_networks", "127.0.0.1/24"); MIN_FE_CONF.put("sys_log_verbose_modules", "org"); diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java index ec0a87ef346970..8861112624f06f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java +++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java @@ -364,12 +364,14 @@ protected int startFEServerWithoutRetry(String runningDir) int feHttpPort = findValidPort(); int feRpcPort = findValidPort(); int feQueryPort = findValidPort(); + int arrowFlightSqlPort = findValidPort(); int feEditLogPort = findValidPort(); Map feConfMap = Maps.newHashMap(); // set additional fe config feConfMap.put("http_port", String.valueOf(feHttpPort)); feConfMap.put("rpc_port", String.valueOf(feRpcPort)); feConfMap.put("query_port", String.valueOf(feQueryPort)); + feConfMap.put("arrow_flight_sql_port", String.valueOf(arrowFlightSqlPort)); feConfMap.put("edit_log_port", String.valueOf(feEditLogPort)); feConfMap.put("tablet_create_timeout_second", "10"); // start fe in "DORIS_HOME/fe/mocked/" @@ -449,10 +451,11 @@ private Backend createBackendWithoutRetry(String beHost, int feRpcPort) throws I int beThriftPort = findValidPort(); int beBrpcPort = findValidPort(); int beHttpPort = findValidPort(); + int beArrowFlightSqlPort = findValidPort(); // start be MockedBackend backend = MockedBackendFactory.createBackend(beHost, beHeartbeatPort, beThriftPort, beBrpcPort, - beHttpPort, new DefaultHeartbeatServiceImpl(beThriftPort, beHttpPort, beBrpcPort), + beHttpPort, beArrowFlightSqlPort, new DefaultHeartbeatServiceImpl(beThriftPort, beHttpPort, beBrpcPort, beArrowFlightSqlPort), new DefaultBeThriftServiceImpl(), new DefaultPBackendServiceImpl()); backend.setFeAddress(new TNetworkAddress("127.0.0.1", feRpcPort)); backend.start(); @@ -471,6 +474,7 @@ beHttpPort, new DefaultHeartbeatServiceImpl(beThriftPort, beHttpPort, beBrpcPort be.setBePort(beThriftPort); be.setHttpPort(beHttpPort); be.setBrpcPort(beBrpcPort); + be.setArrowFlightSqlPort(beArrowFlightSqlPort); Env.getCurrentSystemInfo().addBackend(be); return be; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/UtFrameUtils.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/UtFrameUtils.java index 2e2d53edb7a952..407171a69c7ec9 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/utframe/UtFrameUtils.java +++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/UtFrameUtils.java @@ -193,6 +193,7 @@ private static int startFEServerWithoutRetry(String runningDir) throws EnvVarNot int feHttpPort = findValidPort(); int feRpcPort = findValidPort(); int feQueryPort = findValidPort(); + int arrowFlightSqlPort = findValidPort(); int feEditLogPort = findValidPort(); // start fe in "DORIS_HOME/fe/mocked/" @@ -202,6 +203,7 @@ private static int startFEServerWithoutRetry(String runningDir) throws EnvVarNot feConfMap.put("http_port", String.valueOf(feHttpPort)); feConfMap.put("rpc_port", String.valueOf(feRpcPort)); feConfMap.put("query_port", String.valueOf(feQueryPort)); + feConfMap.put("arrow_flight_sql_port", String.valueOf(arrowFlightSqlPort)); feConfMap.put("edit_log_port", String.valueOf(feEditLogPort)); feConfMap.put("tablet_create_timeout_second", "10"); frontend.init(dorisHome + "/" + runningDir, feConfMap); @@ -278,10 +280,11 @@ private static Backend createBackendWithoutRetry(String beHost, int feRpcPort) t int beThriftPort = findValidPort(); int beBrpcPort = findValidPort(); int beHttpPort = findValidPort(); + int beArrowFlightSqlPort = findValidPort(); // start be MockedBackend backend = MockedBackendFactory.createBackend(beHost, beHeartbeatPort, beThriftPort, beBrpcPort, - beHttpPort, new DefaultHeartbeatServiceImpl(beThriftPort, beHttpPort, beBrpcPort), + beHttpPort, beArrowFlightSqlPort, new DefaultHeartbeatServiceImpl(beThriftPort, beHttpPort, beBrpcPort, beArrowFlightSqlPort), new DefaultBeThriftServiceImpl(), new DefaultPBackendServiceImpl()); backend.setFeAddress(new TNetworkAddress("127.0.0.1", feRpcPort)); backend.start(); @@ -299,6 +302,7 @@ beHttpPort, new DefaultHeartbeatServiceImpl(beThriftPort, beHttpPort, beBrpcPort be.setBePort(beThriftPort); be.setHttpPort(beHttpPort); be.setBrpcPort(beBrpcPort); + be.setArrowFlightSqlPort(beArrowFlightSqlPort); Env.getCurrentSystemInfo().addBackend(be); return be; } diff --git a/fe/pom.xml b/fe/pom.xml index a9ba53de1e2274..3e04562d551f03 100644 --- a/fe/pom.xml +++ b/fe/pom.xml @@ -233,16 +233,17 @@ under the License. 2.18.0 2.0.6 4.0.2 - 4.1.94.Final + + 4.1.96.Final 3.10.6.Final 2.1 - 1.30.0 + 1.56.0 3.32.0 - 3.21.12 + 3.24.3 - 3.21.9 + 3.24.3 com.google.protobuf:protoc:${protoc.artifact.version} io.grpc:protoc-gen-grpc-java:${grpc.version} 3.1.5 @@ -275,7 +276,7 @@ under the License. 1.1.0 3.0.0rc1 0.43.3-public - 9.0.0 + 13.0.0 1.11.1 0.13.1 @@ -313,6 +314,9 @@ under the License. 0.4.0-incubating 3.4.4 + + shade-format-flatbuffers + 1.12.0 @@ -1426,6 +1430,65 @@ under the License. client ${vesoft.client.version} + + io.grpc + grpc-netty + ${grpc.version} + + + io.grpc + grpc-core + ${grpc.version} + + + io.grpc + grpc-context + ${grpc.version} + + + io.netty + netty-tcnative-boringssl-static + + + io.grpc + grpc-api + ${grpc.version} + + + com.google.flatbuffers + flatbuffers-java + ${flatbuffers.version} + + + org.apache.arrow + arrow-vector + ${arrow.version} + + + org.apache.arrow + arrow-memory-netty + ${arrow.version} + + + org.apache.arrow + flight-core + ${arrow.version} + + + org.apache.arrow + flight-sql + ${arrow.version} + + + org.apache.arrow + arrow-memory-core + ${arrow.version} + + + org.apache.arrow + arrow-jdbc + ${arrow.version} + diff --git a/fe_plugins/auditloader/src/main/java/org/apache/doris/plugin/audit/DorisStreamLoader.java b/fe_plugins/auditloader/src/main/java/org/apache/doris/plugin/audit/DorisStreamLoader.java index d2249a3ea7a2d1..2781bcc20734a0 100644 --- a/fe_plugins/auditloader/src/main/java/org/apache/doris/plugin/audit/DorisStreamLoader.java +++ b/fe_plugins/auditloader/src/main/java/org/apache/doris/plugin/audit/DorisStreamLoader.java @@ -145,9 +145,9 @@ public LoadResponse loadBatch(StringBuilder sb, boolean slowLog) { // build request and send to new be location beConn = getConnection(location, label); // send data to be - BufferedOutputStream bos = new BufferedOutputStream(beConn.getOutputStream()); - bos.write(sb.toString().getBytes()); - bos.close(); + try (BufferedOutputStream bos = new BufferedOutputStream(beConn.getOutputStream())) { + bos.write(sb.toString().getBytes()); + } // get respond status = beConn.getResponseCode(); diff --git a/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java b/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java index 55ba457404a01e..279ae688006253 100644 --- a/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java +++ b/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java @@ -343,15 +343,15 @@ public BrokerFileSystem getDistributedFileSystem(String path, Map prop long currentTime = System.currentTimeMillis(); Random random = new Random(currentTime); int randNumber = random.nextInt(10000); - tmpFilePath = "/tmp/." + Long.toString(currentTime) + "_" + Integer.toString(randNumber); - FileOutputStream fileOutputStream = new FileOutputStream(tmpFilePath); - fileOutputStream.write(base64decodedBytes); - fileOutputStream.close(); + tmpFilePath = "/tmp/." + currentTime + "_" + randNumber; + try (FileOutputStream fileOutputStream = new FileOutputStream(tmpFilePath)) { + fileOutputStream.write(base64decodedBytes); + } keytab = tmpFilePath; } else { throw new BrokerException(TBrokerOperationStatusCode.INVALID_ARGUMENT, @@ -948,10 +948,10 @@ public BrokerFileSystem getJuiceFileSystem(String path, Map prop long currentTime = System.currentTimeMillis(); Random random = new Random(currentTime); int randNumber = random.nextInt(10000); - tmpFilePath = "/tmp/." + Long.toString(currentTime) + "_" + Integer.toString(randNumber); - FileOutputStream fileOutputStream = new FileOutputStream(tmpFilePath); - fileOutputStream.write(base64decodedBytes); - fileOutputStream.close(); + tmpFilePath = "/tmp/." + currentTime + "_" + randNumber; + try (FileOutputStream fileOutputStream = new FileOutputStream(tmpFilePath)) { + fileOutputStream.write(base64decodedBytes); + } keytab = tmpFilePath; } else { throw new BrokerException(TBrokerOperationStatusCode.INVALID_ARGUMENT, diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto index a3d14593144e61..878544e74b0bd4 100644 --- a/gensrc/proto/internal_service.proto +++ b/gensrc/proto/internal_service.proto @@ -249,6 +249,16 @@ message PFetchDataResult { optional bool empty_batch = 6; }; +message PFetchArrowFlightSchemaRequest { + optional PUniqueId finst_id = 1; +}; + +message PFetchArrowFlightSchemaResult { + optional PStatus status = 1; + // valid when status is ok + optional bytes schema = 2; +}; + message KeyTuple { repeated string key_column_rep = 1; } @@ -812,5 +822,6 @@ service PBackendService { rpc report_stream_load_status(PReportStreamLoadStatusRequest) returns (PReportStreamLoadStatusResponse); rpc glob(PGlobRequest) returns (PGlobResponse); rpc group_commit_insert(PGroupCommitInsertRequest) returns (PGroupCommitInsertResponse); + rpc fetch_arrow_flight_schema(PFetchArrowFlightSchemaRequest) returns (PFetchArrowFlightSchemaResult); }; diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index e93d466bc800af..0eaa570cdfb68d 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -469,24 +469,24 @@ [['array_remove'], 'ARRAY_VARCHAR', ['ARRAY_VARCHAR', 'VARCHAR'], ''], [['array_remove'], 'ARRAY_STRING', ['ARRAY_STRING', 'STRING'], ''], - [['array_union'], 'ARRAY_BOOLEAN', ['ARRAY_BOOLEAN', 'ARRAY_BOOLEAN'], ''], - [['array_union'], 'ARRAY_TINYINT', ['ARRAY_TINYINT', 'ARRAY_TINYINT'], ''], - [['array_union'], 'ARRAY_SMALLINT', ['ARRAY_SMALLINT', 'ARRAY_SMALLINT'], ''], - [['array_union'], 'ARRAY_INT', ['ARRAY_INT', 'ARRAY_INT'], ''], - [['array_union'], 'ARRAY_BIGINT', ['ARRAY_BIGINT', 'ARRAY_BIGINT'], ''], - [['array_union'], 'ARRAY_LARGEINT', ['ARRAY_LARGEINT', 'ARRAY_LARGEINT'], ''], - [['array_union'], 'ARRAY_FLOAT', ['ARRAY_FLOAT', 'ARRAY_FLOAT'], ''], - [['array_union'], 'ARRAY_DOUBLE', ['ARRAY_DOUBLE', 'ARRAY_DOUBLE'], ''], - [['array_union'], 'ARRAY_DECIMALV2', ['ARRAY_DECIMALV2', 'ARRAY_DECIMALV2'], ''], - [['array_union'], 'ARRAY_DECIMAL32', ['ARRAY_DECIMAL32', 'ARRAY_DECIMAL32'], ''], - [['array_union'], 'ARRAY_DECIMAL64', ['ARRAY_DECIMAL64', 'ARRAY_DECIMAL64'], ''], - [['array_union'], 'ARRAY_DECIMAL128', ['ARRAY_DECIMAL128', 'ARRAY_DECIMAL128'], ''], - [['array_union'], 'ARRAY_DATETIME', ['ARRAY_DATETIME', 'ARRAY_DATETIME'], ''], - [['array_union'], 'ARRAY_DATE', ['ARRAY_DATE', 'ARRAY_DATE'], ''], - [['array_union'], 'ARRAY_DATETIMEV2', ['ARRAY_DATETIMEV2', 'ARRAY_DATETIMEV2'], ''], - [['array_union'], 'ARRAY_DATEV2', ['ARRAY_DATEV2', 'ARRAY_DATEV2'], ''], - [['array_union'], 'ARRAY_VARCHAR', ['ARRAY_VARCHAR', 'ARRAY_VARCHAR'], ''], - [['array_union'], 'ARRAY_STRING', ['ARRAY_STRING', 'ARRAY_STRING'], ''], + [['array_union'], 'ARRAY_BOOLEAN', ['ARRAY_BOOLEAN', 'ARRAY_BOOLEAN', '...'], ''], + [['array_union'], 'ARRAY_TINYINT', ['ARRAY_TINYINT', 'ARRAY_TINYINT', '...'], ''], + [['array_union'], 'ARRAY_SMALLINT', ['ARRAY_SMALLINT', 'ARRAY_SMALLINT', '...'], ''], + [['array_union'], 'ARRAY_INT', ['ARRAY_INT', 'ARRAY_INT', '...'], ''], + [['array_union'], 'ARRAY_BIGINT', ['ARRAY_BIGINT', 'ARRAY_BIGINT', '...'], ''], + [['array_union'], 'ARRAY_LARGEINT', ['ARRAY_LARGEINT', 'ARRAY_LARGEINT', '...'], ''], + [['array_union'], 'ARRAY_FLOAT', ['ARRAY_FLOAT', 'ARRAY_FLOAT', '...'], ''], + [['array_union'], 'ARRAY_DOUBLE', ['ARRAY_DOUBLE', 'ARRAY_DOUBLE', '...'], ''], + [['array_union'], 'ARRAY_DECIMALV2', ['ARRAY_DECIMALV2', 'ARRAY_DECIMALV2', '...'], ''], + [['array_union'], 'ARRAY_DECIMAL32', ['ARRAY_DECIMAL32', 'ARRAY_DECIMAL32', '...'], ''], + [['array_union'], 'ARRAY_DECIMAL64', ['ARRAY_DECIMAL64', 'ARRAY_DECIMAL64', '...'], ''], + [['array_union'], 'ARRAY_DECIMAL128', ['ARRAY_DECIMAL128', 'ARRAY_DECIMAL128', '...'], ''], + [['array_union'], 'ARRAY_DATETIME', ['ARRAY_DATETIME', 'ARRAY_DATETIME', '...'], ''], + [['array_union'], 'ARRAY_DATE', ['ARRAY_DATE', 'ARRAY_DATE', '...'], ''], + [['array_union'], 'ARRAY_DATETIMEV2', ['ARRAY_DATETIMEV2', 'ARRAY_DATETIMEV2', '...'], ''], + [['array_union'], 'ARRAY_DATEV2', ['ARRAY_DATEV2', 'ARRAY_DATEV2', '...'], ''], + [['array_union'], 'ARRAY_VARCHAR', ['ARRAY_VARCHAR', 'ARRAY_VARCHAR', '...'], ''], + [['array_union'], 'ARRAY_STRING', ['ARRAY_STRING', 'ARRAY_STRING', '...'], ''], [['array_apply'], 'ARRAY_BOOLEAN', ['ARRAY_BOOLEAN', 'VARCHAR', 'BOOLEAN'], ''], [['array_apply'], 'ARRAY_TINYINT', ['ARRAY_TINYINT', 'VARCHAR', 'TINYINT'], ''], diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index e5c7b9bb0b600b..6c85c0290cc014 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -824,6 +824,7 @@ struct TFrontendPingFrontendResult { 7: optional i64 lastStartupTime 8: optional list diskInfos 9: optional i64 processUUID + 10: optional i32 arrowFlightSqlPort } struct TPropertyVal { diff --git a/gensrc/thrift/HeartbeatService.thrift b/gensrc/thrift/HeartbeatService.thrift index 7fcf45804d85d3..5a7e47d982b8fb 100644 --- a/gensrc/thrift/HeartbeatService.thrift +++ b/gensrc/thrift/HeartbeatService.thrift @@ -50,6 +50,7 @@ struct TBackendInfo { 6: optional i64 be_start_time // This field will also be uesd to identify a be process 7: optional string be_node_role 8: optional bool is_shutdown + 9: optional Types.TPort arrow_flight_sql_port } struct THeartbeatResult { diff --git a/regression-test/data/datatype_p0/nested_types/test_nested_types_insert_into_with_duplicat_table.out b/regression-test/data/datatype_p0/nested_types/negative_cases/test_nested_types_insert_into_with_dup_table.out similarity index 100% rename from regression-test/data/datatype_p0/nested_types/test_nested_types_insert_into_with_duplicat_table.out rename to regression-test/data/datatype_p0/nested_types/negative_cases/test_nested_types_insert_into_with_dup_table.out diff --git a/regression-test/data/datatype_p0/nested_types/negative_cases/test_nested_types_insert_into_with_unique_table.out b/regression-test/data/datatype_p0/nested_types/negative_cases/test_nested_types_insert_into_with_unique_table.out new file mode 100644 index 00000000000000..2f7adb451860ed --- /dev/null +++ b/regression-test/data/datatype_p0/nested_types/negative_cases/test_nested_types_insert_into_with_unique_table.out @@ -0,0 +1,31 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql_nested_table_mor_c -- +1 + +-- !sql_nested_table_mor2_c -- +1 + +-- !sql_nested_table_map_mor_c -- +1 + +-- !sql_nested_table_array_map_mor_c -- +1 + +-- !sql_nested_table_map_array_mor_c -- +1 + +-- !sql_nested_table_mow_c -- +1 + +-- !sql_nested_table_mow2_c -- +1 + +-- !sql_nested_table_map_mow_c -- +1 + +-- !sql_nested_table_array_map_mow_c -- +1 + +-- !sql_nested_table_map_array_mow_c -- +1 + diff --git a/regression-test/data/jsonb_p0/test_jsonb_load_and_function.out b/regression-test/data/jsonb_p0/test_jsonb_load_and_function.out index 0e0fabb039bf04..29cb48477a787b 100644 --- a/regression-test/data/jsonb_p0/test_jsonb_load_and_function.out +++ b/regression-test/data/jsonb_p0/test_jsonb_load_and_function.out @@ -7461,136 +7461,6 @@ false 30 -9223372036854775808 [null,null,null] 31 18446744073709551615 [null,null,null] --- !select -- -1 \N \N -2 null null -3 true null -4 false null -5 100 null -6 10000 null -7 1000000000 null -8 1152921504606846976 null -9 6.18 null -10 "abcd" null -11 {} null -12 {"k1":"v31","k2":300} "v31" -13 [] null -14 [123,456] null -15 ["abc","def"] null -16 [null,true,false,100,6.18,"abc"] null -17 [{"k1":"v41","k2":400},1,"a",3.14] ["v41"] -18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "v31" -26 \N \N -27 {"k1":"v1","k2":200} "v1" -28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} null -29 12524337771678448270 null -30 -9223372036854775808 null -31 18446744073709551615 null - --- !select -- -1 \N \N -2 null null -3 true null -4 false null -5 100 null -6 10000 null -7 1000000000 null -8 1152921504606846976 null -9 6.18 null -10 "abcd" null -11 {} null -12 {"k1":"v31","k2":300} null -13 [] null -14 [123,456] 456 -15 ["abc","def"] "def" -16 [null,true,false,100,6.18,"abc"] true -17 [{"k1":"v41","k2":400},1,"a",3.14] 1 -18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} null -26 \N \N -27 {"k1":"v1","k2":200} null -28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} null -29 12524337771678448270 null -30 -9223372036854775808 null -31 18446744073709551615 null - --- !select -- -1 \N \N -2 null \N -3 true \N -4 false \N -5 100 \N -6 10000 \N -7 1000000000 \N -8 1152921504606846976 \N -9 6.18 \N -10 "abcd" \N -11 {} \N -12 {"k1":"v31","k2":300} \N -13 [] \N -14 [123,456] \N -15 ["abc","def"] \N -16 [null,true,false,100,6.18,"abc"] \N -17 [{"k1":"v41","k2":400},1,"a",3.14] \N -18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N -26 \N \N -27 {"k1":"v1","k2":200} \N -28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N -29 12524337771678448270 \N -30 -9223372036854775808 \N -31 18446744073709551615 \N - --- !select -- -1 \N \N -2 null null -3 true null -4 false null -5 100 null -6 10000 null -7 1000000000 null -8 1152921504606846976 null -9 6.18 null -10 "abcd" null -11 {} null -12 {"k1":"v31","k2":300} null -13 [] null -14 [123,456] null -15 ["abc","def"] null -16 [null,true,false,100,6.18,"abc"] null -17 [{"k1":"v41","k2":400},1,"a",3.14] null -18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 400 -26 \N \N -27 {"k1":"v1","k2":200} null -28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} null -29 12524337771678448270 null -30 -9223372036854775808 null -31 18446744073709551615 null - --- !select -- -1 \N \N -2 null null -3 true null -4 false null -5 100 null -6 10000 null -7 1000000000 null -8 1152921504606846976 null -9 6.18 null -10 "abcd" null -11 {} null -12 {"k1":"v31","k2":300} null -13 [] null -14 [123,456] null -15 ["abc","def"] null -16 [null,true,false,100,6.18,"abc"] null -17 [{"k1":"v41","k2":400},1,"a",3.14] null -18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "v41" -26 \N \N -27 {"k1":"v1","k2":200} null -28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} null -29 12524337771678448270 null -30 -9223372036854775808 null -31 18446744073709551615 null - -- !sql_json_length -- 1 diff --git a/regression-test/data/load_p0/broker_load/broker_load_with_where.csv b/regression-test/data/load_p0/broker_load/broker_load_with_where.csv new file mode 100644 index 00000000000000..6717da56c7f53b --- /dev/null +++ b/regression-test/data/load_p0/broker_load/broker_load_with_where.csv @@ -0,0 +1,6 @@ +11001,2023-09-01,1,1,10 +11001,2023-09-01,2,1,10 +11001,2023-09-01,1,2,10 +11001,2023-09-01,2,2,10 +11001,2023-09-01,1,3,10 +11001,2023-09-01,2,3,10 \ No newline at end of file diff --git a/regression-test/data/load_p0/broker_load/test_array_load.out b/regression-test/data/load_p0/broker_load/test_array_load.out index f80af4d698660b..816baf4c862353 100644 --- a/regression-test/data/load_p0/broker_load/test_array_load.out +++ b/regression-test/data/load_p0/broker_load/test_array_load.out @@ -65,3 +65,15 @@ -- !select_count -- 6 3 +-- !select -- +100 [1, 2, 3] [32767, 32768, 32769] [65534, 65535, 65536] ["a", "b", "c"] ["hello", "world"] [2022-07-13] [2022-07-13 12:30:00] [0.33, 0.67] [3.1415926, 0.878787878] [4.000000, 5.500000, 6.670000] + +-- !select -- +100 [1, 2, 3] [32767, 32768, 32769] [65534, 65535, 65536] ["a", "b", "c"] ["hello", "world"] [2022-07-13] [2022-07-13 12:30:00] [0.33, 0.67] [3.1415926, 0.878787878] [4.000000, 5.500000, 6.670000] + +-- !select -- +100 [1, 2, 3] [32767, 32768, 32769] [65534, 65535, 65536] ["a", "b", "c"] ["hello", "world"] [2022-07-13] [2022-07-13 12:30:00] [0.33, 0.67] [3.1415926, 0.878787878] [4.000000, 5.500000, 6.670000] + +-- !select -- +100 [1, 2, 3] [32767, 32768, 32769] [65534, 65535, 65536] ["a", "b", "c"] ["hello", "world"] [2022-07-13] [2022-07-13 12:30:00] [0.33, 0.67] [3.1415926, 0.878787878] [4.000000, 5.500000, 6.670000] + diff --git a/regression-test/data/load_p0/stream_load/test_csv_split_line.out b/regression-test/data/load_p0/stream_load/test_csv_split_line.out index 7a97bc931488a6..0b16a8f480f2dc 100644 --- a/regression-test/data/load_p0/stream_load/test_csv_split_line.out +++ b/regression-test/data/load_p0/stream_load/test_csv_split_line.out @@ -4,11 +4,53 @@ -- !sql -- 1000 worldhell 10000000 ello +1111 22131 123123 0000000 2000 wohellhell 200000 ellohell +2222 \N \N \N 3000 worellohell 30000000 elloab 4000 hellwohellhell \N abcdeeelhllo +7777 \N 10001 helloword 114466 0000011445 +33333 00 11111 00000 +44444 00 11111 55555 \N 14455 7711445777 66666 \N \N 113355 77777 0011455 8888 114545 +99999 \N + +-- !sql -- +1 USER 13456 430,431,6418,419,31,341,420,421,7,428,429 0 2023-09-13T09:55:32 +10 \N 1 \N +11 \N \N 2023-09-13T09:57:32 +12 abc 21 1 \N +13 \N 22 1 \N +14 \N \N \N \N \N +15 112 \N 1231 \N \N +16 1 \N 1231 \N \N +2 USER 642836 68,260,257,334,30,218,308,309,31,75 0 2023-09-13T09:57:32 +3 CLASS 366 0 2023-09-13T09:57:32 +4 CLASS 10207 0 2023-09-13T09:57:32 +5 CLASS 111 \N \N +6 USER 1 11 \N \N +7 USER 1 11 \N 2023-09-13T09:57:32 +8 \N \N \N +9 \N 1 \N \N + +-- !sql -- +10 \N 1 \N +11 \N \N 2023-09-13T09:57:32 +12 abc 21 1 \N +3 CLASS 366 0 2023-09-13T09:57:32 +4 CLASS 10207 0 2023-09-13T09:57:32 +5 CLASS 111 \N \N +8 \N \N \N + +-- !sql -- +10 \N 1 \N +11 \N \N 2023-09-13T09:57:32 +12 abc 21 1 \N +3 CLASS 366 0 2023-09-13T09:57:32 +4 CLASS 10207 0 2023-09-13T09:57:32 +5 CLASS 111 \N \N +8 \N \N \N diff --git a/regression-test/data/load_p0/stream_load/test_csv_split_line2.csv b/regression-test/data/load_p0/stream_load/test_csv_split_line2.csv index 04ba509ae49843..94340cebd121a3 100644 --- a/regression-test/data/load_p0/stream_load/test_csv_split_line2.csv +++ b/regression-test/data/load_p0/stream_load/test_csv_split_line2.csv @@ -1,4 +1,7 @@ 1000helloworldhellhello10000000helloello 2000hellowohellhellhello200000helloellohell 3000helloworellohellhello30000000helloelloab -4000hellohellwohellhellhello\Nhelloabcdeeelhllo \ No newline at end of file +4000hellohellwohellhellhello\Nhelloabcdeeelhllo +"1111"hello"22131"hello"123123"hello0000000 +2222hello\Nhello\Nhello\N +7777hellohellohello \ No newline at end of file diff --git a/regression-test/data/load_p0/stream_load/test_csv_split_line3.csv b/regression-test/data/load_p0/stream_load/test_csv_split_line3.csv index 4332f6b90e88e2..f2bb26a8fb422c 100644 --- a/regression-test/data/load_p0/stream_load/test_csv_split_line3.csv +++ b/regression-test/data/load_p0/stream_load/test_csv_split_line3.csv @@ -2,3 +2,6 @@ 55555114455\N114455144551144557711445777 66666114455\N114455\N114455113355 7777711445500114551144558888114455114545 +99999114455114455114455 +33333114455001144551111111445500000 +444441144550011445511111114455 \ No newline at end of file diff --git a/regression-test/data/load_p0/stream_load/test_csv_split_line4.csv b/regression-test/data/load_p0/stream_load/test_csv_split_line4.csv new file mode 100644 index 00000000000000..8956ed41be2d6b --- /dev/null +++ b/regression-test/data/load_p0/stream_load/test_csv_split_line4.csv @@ -0,0 +1,16 @@ +1||USER||13456||430,431,6418,419,31,341,420,421,7,428,429||0||2023-09-13 09:55:32 +2||USER||642836||68,260,257,334,30,218,308,309,31,75||0||2023-09-13 09:57:32 +3||CLASS||366||||0||2023-09-13 09:57:32 +4||CLASS||10207||||0||2023-09-13 09:57:32 +5||CLASS||111|||||| +6||USER||1||11|||| +7||USER||1||11||||2023-09-13 09:57:32 +8|||||||||| +9||||||1|||| +10||||||||1|| +11||||||||||2023-09-13 09:57:32 +12||abc||21||||1|| +13||||||22||1|| +14||\N||\N||\N||\N||\N +15||112||||1231|||| +16||1||||1231|||| \ No newline at end of file diff --git a/regression-test/data/load_p0/stream_load/test_json_load.out b/regression-test/data/load_p0/stream_load/test_json_load.out index ad68ffa1298e83..360c7867d356f9 100644 --- a/regression-test/data/load_p0/stream_load/test_json_load.out +++ b/regression-test/data/load_p0/stream_load/test_json_load.out @@ -205,3 +205,9 @@ John 30 New York {"email":"john@example.com","phone":"+1-123-456-7890"} -- !select22 -- 11324 1321313082437 1678834024274 20230315 {"base_mac_value_null":24,"base_1_value_respiratoryrate":11,"base_3_value_heartrate":51,"base_3_status_onoroutofbed":3,"base_null_count_circulation":84,"base_1_status_onoroutofbed":3,"base_1_value_heartrate":51,"base_3_value_respiratoryrate":11,"base_3_value_bodyactivityenergy":43652,"base_2_value_respiratoryrate":11,"base_2_value_bodyactivityenergy":28831,"base_2_status_onoroutofbed":3,"base_1_value_bodyactivityenergy":56758,"base_2_value_heartrate":51,"tsltype":"properties","datatimestamp":1678834024274,"command":"0105","macaddress":"405EE1805029"} +-- !select -- +200 changsha 3456789 + +-- !select -- +200 changsha 3456789 + diff --git a/regression-test/data/nereids_function_p0/fn_test.dat b/regression-test/data/nereids_function_p0/fn_test.dat index 097a849885d406..f935249c99e608 100644 --- a/regression-test/data/nereids_function_p0/fn_test.dat +++ b/regression-test/data/nereids_function_p0/fn_test.dat @@ -10,4 +10,4 @@ 9;1;10;10;10;10;10;1;1;1;1;1;1;1;1;char11;char21;char31;varchar11;varchar21;varchar31;string1;2012-03-10;2012-03-10;2012-03-10 10:09:10;2012-03-10 10:09:10;2012-03-10 10:09:10;2012-03-10 10:09:10;[1];[1];[10];[10];[10];[10];[10];[1];[2012-03-10,2012-03-10];[2012-03-10 10:09:10];[2012-03-10,2012-03-10];[2012-03-10 10:09:10,2012-03-10 10:09:10,2012-03-10 10:09:10,2012-03-10 10:09:10];[char11,char21,char31];[char11,char21,char31,varchar11,varchar21,varchar31];[char21,char31,varchar11,varchar21,varchar31,string1];[1,1];ST_Point(19.1, 39.1);ST_Point(19.1, 39.1);90.35620117;49.939093;89.4274406433;58.9020987219;0;10;76.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 5 0, 10 4, 4 3, 1 1));{null:1,1:2,1:3};{null:1,0:2,3:3};{null:1,0:2};{null:1,0:2};{null:1,0:2};{null:1,0:2};{null:1,0.0:2};{null:1,0.0:2};{null:1,0.000:2};{null:1,"char0":2};{null:1,"vchar0":2};{null:1,"str0":2};{null:1,2012-03-10:2};{null:1,2012-03-10:2};{1:null,2:1,3:1};{1:null,2:0,3:3};{1:null,2:0};{1:null,2:0};{1:null,2:0};{1:null,2:0};{1:null,2:0.0};{1:null,2:0.0};{1:null,2:0.000};{1:null,2:"char0"};{1:null,2:"vchar0"};{1:null,2:"str0"};{1:null,2:2012-03-10};{1:null,2:2012-03-10} 10;1;11;11;11;11;11;1.1;1.1;1.1;1.1;1.1;1.1;1.1;1.1;char12;char22;char32;varchar12;varchar22;varchar32;string2;2012-03-11;2012-03-11;2012-03-11 11:10:11;2012-03-11 11:10:11;2012-03-11 11:10:11;2012-03-11 11:10:11;[1];[1];[11];[11];[11];[11];[11];[1.1];[2012-03-11,2012-03-11];[2012-03-11 11:10:11];[2012-03-11,2012-03-11];[2012-03-11 11:10:11,2012-03-11 11:10:11,2012-03-11 11:10:11,2012-03-11 11:10:11];[char12,char22,char32];[char12,char22,char32,varchar12,varchar22,varchar32];[char22,char32,varchar12,varchar22,varchar32,string2];[1.1,1.1];ST_Point(17.1, 37.1);ST_Point(17.1, 37.1);90.35620117;59.939093;89.4274406433;58.9020987219;0;11;75.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 8 0, 48 34, 4 10, 1 1));{null:1,1:2,1:3};{null:1,1:2,2:3};{null:1,1:2};{null:1,1:2};{null:1,1:2};{null:1,1:2};{null:1,1.1:2};{null:1,1.1:2};{null:1,1.100:2};{null:1,"char1":2};{null:1,"vchar1":2};{null:1,"str1":2};{null:1,2012-03-11:2};{null:1,2012-03-11:2};{1:null,2:1,3:1};{1:null,2:1,3:2};{1:null,2:1};{1:null,2:1};{1:null,2:1};{1:null,2:1};{1:null,2:1.1};{1:null,2:1.1};{1:null,2:1.100};{1:null,2:"char1"};{1:null,2:"vchar1"};{1:null,2:"str1"};{1:null,2:2012-03-11};{1:null,2:2012-03-11} 11;1;12;12;12;12;12;1.2;1.2;1.2;1.2;1.2;1.2;1.2;1.2;char13;char23;char33;varchar13;varchar23;varchar33;string3;2012-03-12;2012-03-12;2012-03-12 12:11:12;2012-03-12 12:11:12;2012-03-12 12:11:12;2012-03-12 12:11:12;[1];[1];[12];[12];[12];[12];[12];[1.2];[2012-03-12,2012-03-12];[2012-03-12 12:11:12];[2012-03-12,2012-03-12];[2012-03-12 12:11:12,2012-03-12 12:11:12,2012-03-12 12:11:12,2012-03-12 12:11:12];[char13,char23,char33];[char13,char23,char33,varchar13,varchar23,varchar33];[char23,char33,varchar13,varchar23,varchar33,string3];[1.2,1.2];ST_Point(22.1, 22.1);ST_Point(22.1, 22.1);90.35620117;39.939093;89.4274406433;58.9020987219;0;12;100.321;LINESTRING (1 1, 2 2);POLYGON ((1 1, 9 4, 10 4, 4 1, 1 1));{null:1,1:2,1:3};{null:1,2:2,1:3};{null:1,2:2};{null:1,2:2};{null:1,2:2};{null:1,2:2};{null:1,2.2:2};{null:1,2.2:2};{null:1,2.200:2};{null:1,"char2":2};{null:1,"vchar2":2};{null:1,"str2":2};{null:1,2012-03-12:2};{null:1,2012-03-12:2};{1:null,2:1,3:1};{1:null,2:2,3:1};{1:null,2:2};{1:null,2:2};{1:null,2:2};{1:null,2:2};{1:null,2:2.2};{1:null,2:2.2};{1:null,2:2.200};{1:null,2:"char2"};{1:null,2:"vchar2"};{1:null,2:"str2"};{1:null,2:2012-03-12};{1:null,2:2012-03-12} -null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null \ No newline at end of file +null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null;null diff --git a/regression-test/data/nereids_function_p0/scalar_function/Array.out b/regression-test/data/nereids_function_p0/scalar_function/Array.out index 7c63c9abd8bac9..a0a2b958e9ffa9 100644 --- a/regression-test/data/nereids_function_p0/scalar_function/Array.out +++ b/regression-test/data/nereids_function_p0/scalar_function/Array.out @@ -7773,90 +7773,90 @@ true -- !sql_array_union_Char -- \N -["char11", "char21", "char31"] -["char11", "char21", "char31"] -["char11", "char21", "char31"] -["char11", "char21", "char31"] -["char12", "char22", "char32"] -["char12", "char22", "char32"] -["char12", "char22", "char32"] -["char12", "char22", "char32"] +["char12", "char32", "char22"] +["char12", "char32", "char22"] +["char12", "char32", "char22"] +["char12", "char32", "char22"] ["char13", "char23", "char33"] ["char13", "char23", "char33"] ["char13", "char23", "char33"] ["char13", "char23", "char33"] +["char21", "char11", "char31"] +["char21", "char11", "char31"] +["char21", "char11", "char31"] +["char21", "char11", "char31"] -- !sql_array_union_Char_notnull -- -["char11", "char21", "char31"] -["char11", "char21", "char31"] -["char11", "char21", "char31"] -["char11", "char21", "char31"] -["char12", "char22", "char32"] -["char12", "char22", "char32"] -["char12", "char22", "char32"] -["char12", "char22", "char32"] +["char12", "char32", "char22"] +["char12", "char32", "char22"] +["char12", "char32", "char22"] +["char12", "char32", "char22"] ["char13", "char23", "char33"] ["char13", "char23", "char33"] ["char13", "char23", "char33"] ["char13", "char23", "char33"] +["char21", "char11", "char31"] +["char21", "char11", "char31"] +["char21", "char11", "char31"] +["char21", "char11", "char31"] -- !sql_array_union_Varchar -- \N -["char11", "char21", "char31", "varchar11", "varchar21", "varchar31"] -["char11", "char21", "char31", "varchar11", "varchar21", "varchar31"] -["char11", "char21", "char31", "varchar11", "varchar21", "varchar31"] -["char11", "char21", "char31", "varchar11", "varchar21", "varchar31"] -["char12", "char22", "char32", "varchar12", "varchar22", "varchar32"] -["char12", "char22", "char32", "varchar12", "varchar22", "varchar32"] -["char12", "char22", "char32", "varchar12", "varchar22", "varchar32"] -["char12", "char22", "char32", "varchar12", "varchar22", "varchar32"] -["char13", "char23", "char33", "varchar13", "varchar23", "varchar33"] -["char13", "char23", "char33", "varchar13", "varchar23", "varchar33"] -["char13", "char23", "char33", "varchar13", "varchar23", "varchar33"] -["char13", "char23", "char33", "varchar13", "varchar23", "varchar33"] +["varchar11", "varchar31", "char21", "char11", "char31", "varchar21"] +["varchar11", "varchar31", "char21", "char11", "char31", "varchar21"] +["varchar11", "varchar31", "char21", "char11", "char31", "varchar21"] +["varchar11", "varchar31", "char21", "char11", "char31", "varchar21"] +["varchar22", "char12", "char32", "varchar12", "char22", "varchar32"] +["varchar22", "char12", "char32", "varchar12", "char22", "varchar32"] +["varchar22", "char12", "char32", "varchar12", "char22", "varchar32"] +["varchar22", "char12", "char32", "varchar12", "char22", "varchar32"] +["varchar23", "char13", "char23", "char33", "varchar13", "varchar33"] +["varchar23", "char13", "char23", "char33", "varchar13", "varchar33"] +["varchar23", "char13", "char23", "char33", "varchar13", "varchar33"] +["varchar23", "char13", "char23", "char33", "varchar13", "varchar33"] -- !sql_array_union_Varchar_notnull -- -["char11", "char21", "char31", "varchar11", "varchar21", "varchar31"] -["char11", "char21", "char31", "varchar11", "varchar21", "varchar31"] -["char11", "char21", "char31", "varchar11", "varchar21", "varchar31"] -["char11", "char21", "char31", "varchar11", "varchar21", "varchar31"] -["char12", "char22", "char32", "varchar12", "varchar22", "varchar32"] -["char12", "char22", "char32", "varchar12", "varchar22", "varchar32"] -["char12", "char22", "char32", "varchar12", "varchar22", "varchar32"] -["char12", "char22", "char32", "varchar12", "varchar22", "varchar32"] -["char13", "char23", "char33", "varchar13", "varchar23", "varchar33"] -["char13", "char23", "char33", "varchar13", "varchar23", "varchar33"] -["char13", "char23", "char33", "varchar13", "varchar23", "varchar33"] -["char13", "char23", "char33", "varchar13", "varchar23", "varchar33"] +["varchar11", "varchar31", "char21", "char11", "char31", "varchar21"] +["varchar11", "varchar31", "char21", "char11", "char31", "varchar21"] +["varchar11", "varchar31", "char21", "char11", "char31", "varchar21"] +["varchar11", "varchar31", "char21", "char11", "char31", "varchar21"] +["varchar22", "char12", "char32", "varchar12", "char22", "varchar32"] +["varchar22", "char12", "char32", "varchar12", "char22", "varchar32"] +["varchar22", "char12", "char32", "varchar12", "char22", "varchar32"] +["varchar22", "char12", "char32", "varchar12", "char22", "varchar32"] +["varchar23", "char13", "char23", "char33", "varchar13", "varchar33"] +["varchar23", "char13", "char23", "char33", "varchar13", "varchar33"] +["varchar23", "char13", "char23", "char33", "varchar13", "varchar33"] +["varchar23", "char13", "char23", "char33", "varchar13", "varchar33"] -- !sql_array_union_String -- \N -["char21", "char31", "varchar11", "varchar21", "varchar31", "string1"] -["char21", "char31", "varchar11", "varchar21", "varchar31", "string1"] -["char21", "char31", "varchar11", "varchar21", "varchar31", "string1"] -["char21", "char31", "varchar11", "varchar21", "varchar31", "string1"] -["char22", "char32", "varchar12", "varchar22", "varchar32", "string2"] -["char22", "char32", "varchar12", "varchar22", "varchar32", "string2"] -["char22", "char32", "varchar12", "varchar22", "varchar32", "string2"] -["char22", "char32", "varchar12", "varchar22", "varchar32", "string2"] -["char23", "char33", "varchar13", "varchar23", "varchar33", "string3"] -["char23", "char33", "varchar13", "varchar23", "varchar33", "string3"] -["char23", "char33", "varchar13", "varchar23", "varchar33", "string3"] -["char23", "char33", "varchar13", "varchar23", "varchar33", "string3"] +["varchar11", "varchar31", "char21", "char31", "string1", "varchar21"] +["varchar11", "varchar31", "char21", "char31", "string1", "varchar21"] +["varchar11", "varchar31", "char21", "char31", "string1", "varchar21"] +["varchar11", "varchar31", "char21", "char31", "string1", "varchar21"] +["varchar22", "string2", "char32", "varchar12", "char22", "varchar32"] +["varchar22", "string2", "char32", "varchar12", "char22", "varchar32"] +["varchar22", "string2", "char32", "varchar12", "char22", "varchar32"] +["varchar22", "string2", "char32", "varchar12", "char22", "varchar32"] +["varchar23", "char23", "string3", "char33", "varchar13", "varchar33"] +["varchar23", "char23", "string3", "char33", "varchar13", "varchar33"] +["varchar23", "char23", "string3", "char33", "varchar13", "varchar33"] +["varchar23", "char23", "string3", "char33", "varchar13", "varchar33"] -- !sql_array_union_String_notnull -- -["char21", "char31", "varchar11", "varchar21", "varchar31", "string1"] -["char21", "char31", "varchar11", "varchar21", "varchar31", "string1"] -["char21", "char31", "varchar11", "varchar21", "varchar31", "string1"] -["char21", "char31", "varchar11", "varchar21", "varchar31", "string1"] -["char22", "char32", "varchar12", "varchar22", "varchar32", "string2"] -["char22", "char32", "varchar12", "varchar22", "varchar32", "string2"] -["char22", "char32", "varchar12", "varchar22", "varchar32", "string2"] -["char22", "char32", "varchar12", "varchar22", "varchar32", "string2"] -["char23", "char33", "varchar13", "varchar23", "varchar33", "string3"] -["char23", "char33", "varchar13", "varchar23", "varchar33", "string3"] -["char23", "char33", "varchar13", "varchar23", "varchar33", "string3"] -["char23", "char33", "varchar13", "varchar23", "varchar33", "string3"] +["varchar11", "varchar31", "char21", "char31", "string1", "varchar21"] +["varchar11", "varchar31", "char21", "char31", "string1", "varchar21"] +["varchar11", "varchar31", "char21", "char31", "string1", "varchar21"] +["varchar11", "varchar31", "char21", "char31", "string1", "varchar21"] +["varchar22", "string2", "char32", "varchar12", "char22", "varchar32"] +["varchar22", "string2", "char32", "varchar12", "char22", "varchar32"] +["varchar22", "string2", "char32", "varchar12", "char22", "varchar32"] +["varchar22", "string2", "char32", "varchar12", "char22", "varchar32"] +["varchar23", "char23", "string3", "char33", "varchar13", "varchar33"] +["varchar23", "char23", "string3", "char33", "varchar13", "varchar33"] +["varchar23", "char23", "string3", "char33", "varchar13", "varchar33"] +["varchar23", "char23", "string3", "char33", "varchar13", "varchar33"] -- !sql_array_union_DatetimeV2 -- \N diff --git a/regression-test/data/nereids_function_p0/scalar_function/J.out b/regression-test/data/nereids_function_p0/scalar_function/J.out new file mode 100644 index 00000000000000..fa10d720005433 --- /dev/null +++ b/regression-test/data/nereids_function_p0/scalar_function/J.out @@ -0,0 +1,7463 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select -- +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31","k2":300} +13 [] +14 [123,456] +15 ["abc","def"] +16 [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} + +-- !select -- +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31","k2":300} +13 [] +14 [123,456] +15 ["abc","def"] +16 [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N +27 {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 +30 -9223372036854775808 +31 18446744073709551615 + +-- !jsonb_extract_select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" "abcd" +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 12524337771678448270 +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 18446744073709551615 + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} "v31" +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "v31" +26 \N \N +27 {"k1":"v1","k2":200} "v1" +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"k1.a1":"v31","k2":300} +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} "v31" +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] \N +14 [123,456] 123 +15 ["abc","def"] "abc" +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] {"k1":"v41","k2":400} +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] "def" +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] "a" +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] 3.14 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 6.18 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] "abc" +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [{"k1":"v41","k2":400},1,"a",3.14] +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v41","k2":400} +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "a" +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "a" +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v41","k2":400} +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !jsonb_extract_multipath -- +1 \N \N +2 null [null,null,null,null] +3 true [true,null,null,null] +4 false [false,null,null,null] +5 100 [100,null,null,null] +6 10000 [10000,null,null,null] +7 1000000000 [1000000000,null,null,null] +8 1152921504606846976 [1152921504606846976,null,null,null] +9 6.18 [6.18,null,null,null] +10 "abcd" ["abcd",null,null,null] +11 {} [{},{},null,{}] +12 {"k1":"v31","k2":300} [{"k1":"v31","k2":300},{"k1":"v31","k2":300},"v31",{"k1":"v31","k2":300}] +13 [] [[],null,null,null] +14 [123,456] [[123,456],null,null,123] +15 ["abc","def"] [["abc","def"],null,null,"abc"] +16 [null,true,false,100,6.18,"abc"] [[null,true,false,100,6.18,"abc"],null,null,null] +17 [{"k1":"v41","k2":400},1,"a",3.14] [[{"k1":"v41","k2":400},1,"a",3.14],null,null,{"k1":"v41","k2":400}] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [{"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]},{"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]},"v31",{"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]}] +26 \N \N +27 {"k1":"v1","k2":200} [{"k1":"v1","k2":200},{"k1":"v1","k2":200},"v1",{"k1":"v1","k2":200}] +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} [{"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"},{"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"},null,{"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"}] +29 12524337771678448270 [12524337771678448270,null,null,null] +30 -9223372036854775808 [-9223372036854775808,null,null,null] +31 18446744073709551615 [18446744073709551615,null,null,null] + +-- !jsonb_extract_string_select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" abcd +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 12524337771678448270 +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 18446744073709551615 + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} v31 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} v31 +26 \N \N +27 {"k1":"v1","k2":200} v1 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] \N +14 [123,456] 123 +15 ["abc","def"] abc +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] {"k1":"v41","k2":400} +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] def +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] a +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] 3.14 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 6.18 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] abc +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [{"k1":"v41","k2":400},1,"a",3.14] +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v41","k2":400} +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} a +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} a +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v41","k2":400} +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !jsonb_extract_int_select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !jsonb_extract_bigint_select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !jsonb_extract_largeint_select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 12524337771678448270 +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 18446744073709551615 + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !jsonb_extract_double_select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100.0 +6 10000 10000.0 +7 1000000000 1.0E9 +8 1152921504606846976 1.15292150460684698E18 +9 6.18 6.18 +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9.223372036854776E18 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300.0 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300.0 +26 \N \N +27 {"k1":"v1","k2":200} 200.0 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123.0 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456.0 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1.0 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100.0 +17 [{"k1":"v41","k2":400},1,"a",3.14] 3.14 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 6.18 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1.0 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1.0 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !jsonb_extract_bool_select -- +1 \N \N +2 null \N +3 true true +4 false false +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !jsonb_extract_isnull_select -- +1 \N \N +2 null true +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} false +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} false +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} false +12 {"k1":"v31","k2":300} false +13 [] \N +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !jsonb_exists_path_select -- +1 \N \N +2 null true +3 true true +4 false true +5 100 true +6 10000 true +7 1000000000 true +8 1152921504606846976 true +9 6.18 true +10 "abcd" true +11 {} true +12 {"k1":"v31","k2":300} true +13 [] true +14 [123,456] true +15 ["abc","def"] true +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} true +29 12524337771678448270 true +30 -9223372036854775808 true +31 18446744073709551615 true + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} true +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} true +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} true +12 {"k1":"v31","k2":300} true +13 [] false +14 [123,456] true +15 ["abc","def"] true +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} true +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] true +15 ["abc","def"] true +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !jsonb_type_select -- +1 \N \N +2 null null +3 true bool +4 false bool +5 100 int +6 10000 int +7 1000000000 int +8 1152921504606846976 bigint +9 6.18 double +10 "abcd" string +11 {} object +12 {"k1":"v31","k2":300} object +13 [] array +14 [123,456] array +15 ["abc","def"] array +16 [null,true,false,100,6.18,"abc"] array +17 [{"k1":"v41","k2":400},1,"a",3.14] array +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} object +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} object +29 12524337771678448270 largeint +30 -9223372036854775808 bigint +31 18446744073709551615 largeint + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} string +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} string +26 \N \N +27 {"k1":"v1","k2":200} string +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} int +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} int +26 \N \N +27 {"k1":"v1","k2":200} int +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} object +12 {"k1":"v31","k2":300} object +13 [] \N +14 [123,456] int +15 ["abc","def"] string +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] object +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} object +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} object +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] int +15 ["abc","def"] string +16 [null,true,false,100,6.18,"abc"] bool +17 [{"k1":"v41","k2":400},1,"a",3.14] int +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] bool +17 [{"k1":"v41","k2":400},1,"a",3.14] string +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] int +17 [{"k1":"v41","k2":400},1,"a",3.14] double +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] double +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] string +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} array +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} int +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} string +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} double +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} double +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} double +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} string +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} int +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} double +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !cast_from_select -- +1 \N \N +2 null \N +3 true true +4 false false +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100.0 +6 10000 10000.0 +7 1000000000 1.0E9 +8 1152921504606846976 1.15292150460684698E18 +9 6.18 6.18 +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9.223372036854776E18 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" "abcd" +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 12524337771678448270 +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 18446744073709551615 + +-- !cast_to_select -- +1 \N \N +2 null \N +3 true true +4 false false +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1.15292150460685e+18 +9 6.18 6.18 +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9.22337203685478e+18 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" "abcd" +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 12524337771678448270 +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 18446744073709551615 + +-- !select -- +\N + +-- !select -- +null + +-- !select -- +true + +-- !select -- +false + +-- !select -- +100 + +-- !select -- +10000 + +-- !select -- +1000000000 + +-- !select -- +1152921504606846976 + +-- !select -- +6.18 + +-- !select -- +"abcd" + +-- !select -- +{} + +-- !select -- +{"k1":"v31","k2":300} + +-- !select -- +[] + +-- !select -- +[123,456] + +-- !select -- +["abc","def"] + +-- !select -- +[null,true,false,100,6.18,"abc"] + +-- !select -- +[{"k1":"v41","k2":400},1,"a",3.14] + +-- !select -- +{"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +1 \N +2 1 +3 1 +4 1 +5 1 +6 1 +7 1 +8 1 +9 1 +10 1 +11 1 +12 1 +13 1 +14 1 +15 1 +16 1 +17 1 +18 1 +26 \N +27 1 +28 1 +29 1 +30 1 +31 1 + +-- !select -- +1 + +-- !select -- +0 + +-- !select -- +\N + +-- !select -- +1 \N \N +2 null null +3 true null +4 false null +5 100 null +6 10000 null +7 1000000000 null +8 1152921504606846976 null +9 6.18 null +10 "abcd" null +11 {} null +12 {"k1":"v31","k2":300} "v31" +13 [] null +14 [123,456] null +15 ["abc","def"] null +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] ["v41"] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "v31" +26 \N \N +27 {"k1":"v1","k2":200} "v1" +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} null +29 12524337771678448270 null +30 -9223372036854775808 null +31 18446744073709551615 null + +-- !select -- +1 \N \N +2 null [null,null] +3 true [null,null] +4 false [null,null] +5 100 [null,null] +6 10000 [null,null] +7 1000000000 [null,null] +8 1152921504606846976 [null,null] +9 6.18 [null,null] +10 "abcd" [null,null] +11 {} [null,null] +12 {"k1":"v31","k2":300} [300,null] +13 [] [null,null] +14 [123,456] [null,456] +15 ["abc","def"] [null,"def"] +16 [null,true,false,100,6.18,"abc"] [null,true] +17 [{"k1":"v41","k2":400},1,"a",3.14] [[400],1] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [300,null] +26 \N \N +27 {"k1":"v1","k2":200} [200,null] +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} [null,null] +29 12524337771678448270 [null,null] +30 -9223372036854775808 [null,null] +31 18446744073709551615 [null,null] + +-- !select -- +1 \N \N +2 null [null,null] +3 true [null,null] +4 false [null,null] +5 100 [null,null] +6 10000 [null,null] +7 1000000000 [null,null] +8 1152921504606846976 [null,null] +9 6.18 [null,null] +10 "abcd" [null,null] +11 {} [null,null] +12 {"k1":"v31","k2":300} [300,null] +13 [] [null,null] +14 [123,456] [null,null] +15 ["abc","def"] [null,null] +16 [null,true,false,100,6.18,"abc"] [null,null] +17 [{"k1":"v41","k2":400},1,"a",3.14] [[400],null] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [300,null] +26 \N \N +27 {"k1":"v1","k2":200} [200,null] +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} [null,null] +29 12524337771678448270 [null,null] +30 -9223372036854775808 [null,null] +31 18446744073709551615 [null,null] + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null [null,null,null] +3 true [null,null,null] +4 false [null,null,null] +5 100 [null,null,null] +6 10000 [null,null,null] +7 1000000000 [null,null,null] +8 1152921504606846976 [null,null,null] +9 6.18 [null,null,null] +10 "abcd" [null,null,null] +11 {} [null,null,null] +12 {"k1":"v31","k2":300} [null,null,null] +13 [] [null,null,null] +14 [123,456] [null,null,null] +15 ["abc","def"] [null,null,null] +16 [null,true,false,100,6.18,"abc"] [null,null,null] +17 [{"k1":"v41","k2":400},1,"a",3.14] [null,null,null] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} ["v41",400,"a"] +26 \N \N +27 {"k1":"v1","k2":200} [null,null,null] +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} [null,null,null] +29 12524337771678448270 [null,null,null] +30 -9223372036854775808 [null,null,null] +31 18446744073709551615 [null,null,null] + diff --git a/regression-test/data/nereids_function_p0/scalar_function/test_jsonb.csv b/regression-test/data/nereids_function_p0/scalar_function/test_jsonb.csv new file mode 100644 index 00000000000000..3a33f80e5b6cd7 --- /dev/null +++ b/regression-test/data/nereids_function_p0/scalar_function/test_jsonb.csv @@ -0,0 +1,25 @@ +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31", "k2": 300} +13 [] +14 [123, 456] +15 ["abc", "def"] +16 [null, true, false, 100, 6.18, "abc"] +17 [{"k1":"v41", "k2": 400}, 1, "a", 3.14] +18 {"k1":"v31", "k2": 300, "a1": [{"k1":"v41", "k2": 400}, 1, "a", 3.14]} +19 '' +20 'abc' +21 abc +22 100x +23 6.a8 +24 {x +25 [123, abc] diff --git a/regression-test/data/nereids_p0/json_p0/test_json.csv b/regression-test/data/nereids_p0/json_p0/test_json.csv new file mode 100644 index 00000000000000..3a33f80e5b6cd7 --- /dev/null +++ b/regression-test/data/nereids_p0/json_p0/test_json.csv @@ -0,0 +1,25 @@ +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31", "k2": 300} +13 [] +14 [123, 456] +15 ["abc", "def"] +16 [null, true, false, 100, 6.18, "abc"] +17 [{"k1":"v41", "k2": 400}, 1, "a", 3.14] +18 {"k1":"v31", "k2": 300, "a1": [{"k1":"v41", "k2": 400}, 1, "a", 3.14]} +19 '' +20 'abc' +21 abc +22 100x +23 6.a8 +24 {x +25 [123, abc] diff --git a/regression-test/data/nereids_p0/json_p0/test_json_load_and_function.out b/regression-test/data/nereids_p0/json_p0/test_json_load_and_function.out new file mode 100644 index 00000000000000..5fdb5342cbee3b --- /dev/null +++ b/regression-test/data/nereids_p0/json_p0/test_json_load_and_function.out @@ -0,0 +1,5669 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select -- +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31","k2":300} +13 [] +14 [123,456] +15 ["abc","def"] +16 [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} + +-- !select -- +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31","k2":300} +13 [] +14 [123,456] +15 ["abc","def"] +16 [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N +27 {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 +30 -9223372036854775808 +31 18446744073709551615 + +-- !select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" "abcd" +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 12524337771678448270 +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 18446744073709551615 + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} "v31" +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "v31" +26 \N \N +27 {"k1":"v1","k2":200} "v1" +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"k1.a1":"v31","k2":300} +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} "v31" +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] \N +14 [123,456] 123 +15 ["abc","def"] "abc" +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] {"k1":"v41","k2":400} +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] "def" +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] "a" +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] 3.14 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 6.18 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] "abc" +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [{"k1":"v41","k2":400},1,"a",3.14] +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v41","k2":400} +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "a" +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "a" +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" abcd +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 12524337771678448270 +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 18446744073709551615 + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} v31 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} v31 +26 \N \N +27 {"k1":"v1","k2":200} v1 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] \N +14 [123,456] 123 +15 ["abc","def"] abc +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] {"k1":"v41","k2":400} +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] def +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] a +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] 3.14 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 6.18 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] abc +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [{"k1":"v41","k2":400},1,"a",3.14] +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v41","k2":400} +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} a +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !json_extract_largeint_select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 12524337771678448270 +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 18446744073709551615 + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100.0 +6 10000 10000.0 +7 1000000000 1.0E9 +8 1152921504606846976 1.15292150460684698E18 +9 6.18 6.18 +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9.223372036854776E18 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300.0 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300.0 +26 \N \N +27 {"k1":"v1","k2":200} 200.0 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123.0 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456.0 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1.0 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100.0 +17 [{"k1":"v41","k2":400},1,"a",3.14] 3.14 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 6.18 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1.0 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true true +4 false false +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null true +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} false +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} false +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} false +12 {"k1":"v31","k2":300} false +13 [] \N +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null true +3 true true +4 false true +5 100 true +6 10000 true +7 1000000000 true +8 1152921504606846976 true +9 6.18 true +10 "abcd" true +11 {} true +12 {"k1":"v31","k2":300} true +13 [] true +14 [123,456] true +15 ["abc","def"] true +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} true +29 12524337771678448270 true +30 -9223372036854775808 true +31 18446744073709551615 true + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} true +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} true +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} true +12 {"k1":"v31","k2":300} true +13 [] false +14 [123,456] true +15 ["abc","def"] true +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} true +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] true +15 ["abc","def"] true +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null null +3 true bool +4 false bool +5 100 int +6 10000 int +7 1000000000 int +8 1152921504606846976 bigint +9 6.18 double +10 "abcd" string +11 {} object +12 {"k1":"v31","k2":300} object +13 [] array +14 [123,456] array +15 ["abc","def"] array +16 [null,true,false,100,6.18,"abc"] array +17 [{"k1":"v41","k2":400},1,"a",3.14] array +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} object +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} object +29 12524337771678448270 largeint +30 -9223372036854775808 bigint +31 18446744073709551615 largeint + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} string +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} string +26 \N \N +27 {"k1":"v1","k2":200} string +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} int +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} int +26 \N \N +27 {"k1":"v1","k2":200} int +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} object +12 {"k1":"v31","k2":300} object +13 [] \N +14 [123,456] int +15 ["abc","def"] string +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] object +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} object +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} object +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] int +15 ["abc","def"] string +16 [null,true,false,100,6.18,"abc"] bool +17 [{"k1":"v41","k2":400},1,"a",3.14] int +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] bool +17 [{"k1":"v41","k2":400},1,"a",3.14] string +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] int +17 [{"k1":"v41","k2":400},1,"a",3.14] double +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] double +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] string +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} array +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} int +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} string +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} double +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true true +4 false false +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100.0 +6 10000 10000.0 +7 1000000000 1.0E9 +8 1152921504606846976 1.15292150460684698E18 +9 6.18 6.18 +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9.223372036854776E18 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" "abcd" +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 12524337771678448270 +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 18446744073709551615 + +-- !select -- +1 \N \N +2 null \N +3 true true +4 false false +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1.15292150460685e+18 +9 6.18 6.18 +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9.22337203685478e+18 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" "abcd" +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 12524337771678448270 +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 18446744073709551615 + +-- !select -- +\N + +-- !select -- +null + +-- !select -- +true + +-- !select -- +false + +-- !select -- +100 + +-- !select -- +10000 + +-- !select -- +1000000000 + +-- !select -- +1152921504606846976 + +-- !select -- +6.18 + +-- !select -- +"abcd" + +-- !select -- +{} + +-- !select -- +{"k1":"v31","k2":300} + +-- !select -- +[] + +-- !select -- +[123,456] + +-- !select -- +["abc","def"] + +-- !select -- +[null,true,false,100,6.18,"abc"] + +-- !select -- +[{"k1":"v41","k2":400},1,"a",3.14] + +-- !select -- +{"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +1 \N +2 1 +3 1 +4 1 +5 1 +6 1 +7 1 +8 1 +9 1 +10 1 +11 1 +12 1 +13 1 +14 1 +15 1 +16 1 +17 1 +18 1 +26 \N +27 1 +28 1 +29 1 +30 1 +31 1 + +-- !select -- +1 + +-- !select -- +0 + +-- !select -- +\N + +-- !select -- +1 \N \N +2 null null +3 true null +4 false null +5 100 null +6 10000 null +7 1000000000 null +8 1152921504606846976 null +9 6.18 null +10 "abcd" null +11 {} null +12 {"k1":"v31","k2":300} "v31" +13 [] null +14 [123,456] null +15 ["abc","def"] null +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] ["v41"] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "v31" +26 \N \N +27 {"k1":"v1","k2":200} "v1" +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} null +29 12524337771678448270 null +30 -9223372036854775808 null +31 18446744073709551615 null + +-- !select -- +1 \N \N +2 null [null,null] +3 true [null,null] +4 false [null,null] +5 100 [null,null] +6 10000 [null,null] +7 1000000000 [null,null] +8 1152921504606846976 [null,null] +9 6.18 [null,null] +10 "abcd" [null,null] +11 {} [null,null] +12 {"k1":"v31","k2":300} [300,null] +13 [] [null,null] +14 [123,456] [null,456] +15 ["abc","def"] [null,"def"] +16 [null,true,false,100,6.18,"abc"] [null,true] +17 [{"k1":"v41","k2":400},1,"a",3.14] [[400],1] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [300,null] +26 \N \N +27 {"k1":"v1","k2":200} [200,null] +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} [null,null] +29 12524337771678448270 [null,null] +30 -9223372036854775808 [null,null] +31 18446744073709551615 [null,null] + +-- !select -- +1 \N \N +2 null [null,null] +3 true [null,null] +4 false [null,null] +5 100 [null,null] +6 10000 [null,null] +7 1000000000 [null,null] +8 1152921504606846976 [null,null] +9 6.18 [null,null] +10 "abcd" [null,null] +11 {} [null,null] +12 {"k1":"v31","k2":300} [300,null] +13 [] [null,null] +14 [123,456] [null,null] +15 ["abc","def"] [null,null] +16 [null,true,false,100,6.18,"abc"] [null,null] +17 [{"k1":"v41","k2":400},1,"a",3.14] [[400],null] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [300,null] +26 \N \N +27 {"k1":"v1","k2":200} [200,null] +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} [null,null] +29 12524337771678448270 [null,null] +30 -9223372036854775808 [null,null] +31 18446744073709551615 [null,null] + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null [null,null,null] +3 true [null,null,null] +4 false [null,null,null] +5 100 [null,null,null] +6 10000 [null,null,null] +7 1000000000 [null,null,null] +8 1152921504606846976 [null,null,null] +9 6.18 [null,null,null] +10 "abcd" [null,null,null] +11 {} [null,null,null] +12 {"k1":"v31","k2":300} [null,null,null] +13 [] [null,null,null] +14 [123,456] [null,null,null] +15 ["abc","def"] [null,null,null] +16 [null,true,false,100,6.18,"abc"] [null,null,null] +17 [{"k1":"v41","k2":400},1,"a",3.14] [null,null,null] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} ["v41",400,"a"] +26 \N \N +27 {"k1":"v1","k2":200} [null,null,null] +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} [null,null,null] +29 12524337771678448270 [null,null,null] +30 -9223372036854775808 [null,null,null] +31 18446744073709551615 [null,null,null] + diff --git a/regression-test/data/nereids_p0/json_p0/test_json_load_unique_key_and_function.out b/regression-test/data/nereids_p0/json_p0/test_json_load_unique_key_and_function.out new file mode 100644 index 00000000000000..f63959c2e02aa3 --- /dev/null +++ b/regression-test/data/nereids_p0/json_p0/test_json_load_unique_key_and_function.out @@ -0,0 +1,4282 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select -- +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31","k2":300} +13 [] +14 [123,456] +15 ["abc","def"] +16 [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} + +-- !select -- +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31","k2":300} +13 [] +14 [123,456] +15 ["abc","def"] +16 [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N +27 {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} + +-- !select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" "abcd" +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} "v31" +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "v31" +26 \N \N +27 {"k1":"v1","k2":200} "v1" +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"k1.a1":"v31","k2":300} + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} "v31" + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] \N +14 [123,456] 123 +15 ["abc","def"] "abc" +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] {"k1":"v41","k2":400} +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] "def" +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] "a" +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] 3.14 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 6.18 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] "abc" +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [{"k1":"v41","k2":400},1,"a",3.14] +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v41","k2":400} +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "a" +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "a" +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" abcd +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} v31 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} v31 +26 \N \N +27 {"k1":"v1","k2":200} v1 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] \N +14 [123,456] 123 +15 ["abc","def"] abc +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] {"k1":"v41","k2":400} +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] def +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] a +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] 3.14 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 6.18 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] abc +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [{"k1":"v41","k2":400},1,"a",3.14] +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v41","k2":400} +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} a +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100.0 +6 10000 10000.0 +7 1000000000 1.0E9 +8 1152921504606846976 1.15292150460684698E18 +9 6.18 6.18 +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300.0 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300.0 +26 \N \N +27 {"k1":"v1","k2":200} 200.0 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123.0 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456.0 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1.0 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100.0 +17 [{"k1":"v41","k2":400},1,"a",3.14] 3.14 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 6.18 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1.0 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true true +4 false false +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null true +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} false +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} false +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} false +12 {"k1":"v31","k2":300} false +13 [] \N +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null true +3 true true +4 false true +5 100 true +6 10000 true +7 1000000000 true +8 1152921504606846976 true +9 6.18 true +10 "abcd" true +11 {} true +12 {"k1":"v31","k2":300} true +13 [] true +14 [123,456] true +15 ["abc","def"] true +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} true + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} true +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} true +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} true +12 {"k1":"v31","k2":300} true +13 [] false +14 [123,456] true +15 ["abc","def"] true +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} true + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] true +15 ["abc","def"] true +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null null +3 true bool +4 false bool +5 100 int +6 10000 int +7 1000000000 int +8 1152921504606846976 bigint +9 6.18 double +10 "abcd" string +11 {} object +12 {"k1":"v31","k2":300} object +13 [] array +14 [123,456] array +15 ["abc","def"] array +16 [null,true,false,100,6.18,"abc"] array +17 [{"k1":"v41","k2":400},1,"a",3.14] array +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} object +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} object + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} string +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} string +26 \N \N +27 {"k1":"v1","k2":200} string +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} int +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} int +26 \N \N +27 {"k1":"v1","k2":200} int +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} object +12 {"k1":"v31","k2":300} object +13 [] \N +14 [123,456] int +15 ["abc","def"] string +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] object +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} object +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} object + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] int +15 ["abc","def"] string +16 [null,true,false,100,6.18,"abc"] bool +17 [{"k1":"v41","k2":400},1,"a",3.14] int +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] bool +17 [{"k1":"v41","k2":400},1,"a",3.14] string +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] int +17 [{"k1":"v41","k2":400},1,"a",3.14] double +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] double +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] string +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} array +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} int +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} string +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} double +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true true +4 false false +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100.0 +6 10000 10000.0 +7 1000000000 1.0E9 +8 1152921504606846976 1.15292150460684698E18 +9 6.18 6.18 +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" "abcd" +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} + +-- !select -- +1 \N \N +2 null \N +3 true true +4 false false +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1.15292150460685e+18 +9 6.18 6.18 +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" "abcd" +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} + +-- !select -- +\N + +-- !select -- +null + +-- !select -- +true + +-- !select -- +false + +-- !select -- +100 + +-- !select -- +10000 + +-- !select -- +1000000000 + +-- !select -- +1152921504606846976 + +-- !select -- +6.18 + +-- !select -- +"abcd" + +-- !select -- +{} + +-- !select -- +{"k1":"v31","k2":300} + +-- !select -- +[] + +-- !select -- +[123,456] + +-- !select -- +["abc","def"] + +-- !select -- +[null,true,false,100,6.18,"abc"] + +-- !select -- +[{"k1":"v41","k2":400},1,"a",3.14] + +-- !select -- +{"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + diff --git a/regression-test/data/nereids_p0/json_p0/test_json_unique_key.csv b/regression-test/data/nereids_p0/json_p0/test_json_unique_key.csv new file mode 100644 index 00000000000000..88cc69278673c5 --- /dev/null +++ b/regression-test/data/nereids_p0/json_p0/test_json_unique_key.csv @@ -0,0 +1,75 @@ +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31", "k2": 300} +13 [] +14 [123, 456] +15 ["abc", "def"] +16 [null, true, false, 100, 6.18, "abc"] +17 [{"k1":"v41", "k2": 400}, 1, "a", 3.14] +18 {"k1":"v31", "k2": 300, "a1": [{"k1":"v41", "k2": 400}, 1, "a", 3.14]} +19 '' +20 'abc' +21 abc +22 100x +23 6.a8 +24 {x +25 [123, abc] +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31", "k2": 300} +13 [] +14 [123, 456] +15 ["abc", "def"] +16 [null, true, false, 100, 6.18, "abc"] +17 [{"k1":"v41", "k2": 400}, 1, "a", 3.14] +18 {"k1":"v31", "k2": 300, "a1": [{"k1":"v41", "k2": 400}, 1, "a", 3.14]} +19 '' +20 'abc' +21 abc +22 100x +23 6.a8 +24 {x +25 [123, abc] +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31", "k2": 300} +13 [] +14 [123, 456] +15 ["abc", "def"] +16 [null, true, false, 100, 6.18, "abc"] +17 [{"k1":"v41", "k2": 400}, 1, "a", 3.14] +18 {"k1":"v31", "k2": 300, "a1": [{"k1":"v41", "k2": 400}, 1, "a", 3.14]} +19 '' +20 'abc' +21 abc +22 100x +23 6.a8 +24 {x +25 [123, abc] diff --git a/regression-test/data/nereids_p0/jsonb_p0/test_jsonb.csv b/regression-test/data/nereids_p0/jsonb_p0/test_jsonb.csv new file mode 100644 index 00000000000000..3a33f80e5b6cd7 --- /dev/null +++ b/regression-test/data/nereids_p0/jsonb_p0/test_jsonb.csv @@ -0,0 +1,25 @@ +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31", "k2": 300} +13 [] +14 [123, 456] +15 ["abc", "def"] +16 [null, true, false, 100, 6.18, "abc"] +17 [{"k1":"v41", "k2": 400}, 1, "a", 3.14] +18 {"k1":"v31", "k2": 300, "a1": [{"k1":"v41", "k2": 400}, 1, "a", 3.14]} +19 '' +20 'abc' +21 abc +22 100x +23 6.a8 +24 {x +25 [123, abc] diff --git a/regression-test/data/nereids_p0/jsonb_p0/test_jsonb_load_and_function.out b/regression-test/data/nereids_p0/jsonb_p0/test_jsonb_load_and_function.out new file mode 100644 index 00000000000000..fa10d720005433 --- /dev/null +++ b/regression-test/data/nereids_p0/jsonb_p0/test_jsonb_load_and_function.out @@ -0,0 +1,7463 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select -- +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31","k2":300} +13 [] +14 [123,456] +15 ["abc","def"] +16 [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} + +-- !select -- +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31","k2":300} +13 [] +14 [123,456] +15 ["abc","def"] +16 [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N +27 {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 +30 -9223372036854775808 +31 18446744073709551615 + +-- !jsonb_extract_select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" "abcd" +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 12524337771678448270 +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 18446744073709551615 + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} "v31" +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "v31" +26 \N \N +27 {"k1":"v1","k2":200} "v1" +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"k1.a1":"v31","k2":300} +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} "v31" +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] \N +14 [123,456] 123 +15 ["abc","def"] "abc" +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] {"k1":"v41","k2":400} +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] "def" +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] "a" +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] 3.14 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 6.18 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] "abc" +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [{"k1":"v41","k2":400},1,"a",3.14] +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v41","k2":400} +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "a" +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "a" +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v41","k2":400} +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !jsonb_extract_multipath -- +1 \N \N +2 null [null,null,null,null] +3 true [true,null,null,null] +4 false [false,null,null,null] +5 100 [100,null,null,null] +6 10000 [10000,null,null,null] +7 1000000000 [1000000000,null,null,null] +8 1152921504606846976 [1152921504606846976,null,null,null] +9 6.18 [6.18,null,null,null] +10 "abcd" ["abcd",null,null,null] +11 {} [{},{},null,{}] +12 {"k1":"v31","k2":300} [{"k1":"v31","k2":300},{"k1":"v31","k2":300},"v31",{"k1":"v31","k2":300}] +13 [] [[],null,null,null] +14 [123,456] [[123,456],null,null,123] +15 ["abc","def"] [["abc","def"],null,null,"abc"] +16 [null,true,false,100,6.18,"abc"] [[null,true,false,100,6.18,"abc"],null,null,null] +17 [{"k1":"v41","k2":400},1,"a",3.14] [[{"k1":"v41","k2":400},1,"a",3.14],null,null,{"k1":"v41","k2":400}] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [{"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]},{"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]},"v31",{"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]}] +26 \N \N +27 {"k1":"v1","k2":200} [{"k1":"v1","k2":200},{"k1":"v1","k2":200},"v1",{"k1":"v1","k2":200}] +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} [{"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"},{"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"},null,{"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"}] +29 12524337771678448270 [12524337771678448270,null,null,null] +30 -9223372036854775808 [-9223372036854775808,null,null,null] +31 18446744073709551615 [18446744073709551615,null,null,null] + +-- !jsonb_extract_string_select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" abcd +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 12524337771678448270 +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 18446744073709551615 + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} v31 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} v31 +26 \N \N +27 {"k1":"v1","k2":200} v1 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] \N +14 [123,456] 123 +15 ["abc","def"] abc +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] {"k1":"v41","k2":400} +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] def +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] a +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] 3.14 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 6.18 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] abc +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [{"k1":"v41","k2":400},1,"a",3.14] +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v41","k2":400} +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} a +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} a +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v41","k2":400} +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !jsonb_extract_int_select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !jsonb_extract_bigint_select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !jsonb_extract_largeint_select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 12524337771678448270 +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 18446744073709551615 + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !jsonb_extract_double_select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100.0 +6 10000 10000.0 +7 1000000000 1.0E9 +8 1152921504606846976 1.15292150460684698E18 +9 6.18 6.18 +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9.223372036854776E18 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300.0 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300.0 +26 \N \N +27 {"k1":"v1","k2":200} 200.0 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123.0 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456.0 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1.0 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100.0 +17 [{"k1":"v41","k2":400},1,"a",3.14] 3.14 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 6.18 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1.0 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1.0 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !jsonb_extract_bool_select -- +1 \N \N +2 null \N +3 true true +4 false false +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !jsonb_extract_isnull_select -- +1 \N \N +2 null true +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} false +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} false +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} false +12 {"k1":"v31","k2":300} false +13 [] \N +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !jsonb_exists_path_select -- +1 \N \N +2 null true +3 true true +4 false true +5 100 true +6 10000 true +7 1000000000 true +8 1152921504606846976 true +9 6.18 true +10 "abcd" true +11 {} true +12 {"k1":"v31","k2":300} true +13 [] true +14 [123,456] true +15 ["abc","def"] true +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} true +29 12524337771678448270 true +30 -9223372036854775808 true +31 18446744073709551615 true + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} true +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} true +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} true +12 {"k1":"v31","k2":300} true +13 [] false +14 [123,456] true +15 ["abc","def"] true +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} true +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] true +15 ["abc","def"] true +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false +29 12524337771678448270 false +30 -9223372036854775808 false +31 18446744073709551615 false + +-- !jsonb_type_select -- +1 \N \N +2 null null +3 true bool +4 false bool +5 100 int +6 10000 int +7 1000000000 int +8 1152921504606846976 bigint +9 6.18 double +10 "abcd" string +11 {} object +12 {"k1":"v31","k2":300} object +13 [] array +14 [123,456] array +15 ["abc","def"] array +16 [null,true,false,100,6.18,"abc"] array +17 [{"k1":"v41","k2":400},1,"a",3.14] array +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} object +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} object +29 12524337771678448270 largeint +30 -9223372036854775808 bigint +31 18446744073709551615 largeint + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} string +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} string +26 \N \N +27 {"k1":"v1","k2":200} string +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} int +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} int +26 \N \N +27 {"k1":"v1","k2":200} int +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} object +12 {"k1":"v31","k2":300} object +13 [] \N +14 [123,456] int +15 ["abc","def"] string +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] object +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} object +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} object +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] int +15 ["abc","def"] string +16 [null,true,false,100,6.18,"abc"] bool +17 [{"k1":"v41","k2":400},1,"a",3.14] int +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] bool +17 [{"k1":"v41","k2":400},1,"a",3.14] string +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] int +17 [{"k1":"v41","k2":400},1,"a",3.14] double +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] double +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] string +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} array +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} int +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} string +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} double +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} double +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} double +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} string +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} int +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} double +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !cast_from_select -- +1 \N \N +2 null \N +3 true true +4 false false +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100.0 +6 10000 10000.0 +7 1000000000 1.0E9 +8 1152921504606846976 1.15292150460684698E18 +9 6.18 6.18 +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9.223372036854776E18 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" "abcd" +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 12524337771678448270 +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 18446744073709551615 + +-- !cast_to_select -- +1 \N \N +2 null \N +3 true true +4 false false +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1.15292150460685e+18 +9 6.18 6.18 +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 -9.22337203685478e+18 +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" "abcd" +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} +29 12524337771678448270 12524337771678448270 +30 -9223372036854775808 -9223372036854775808 +31 18446744073709551615 18446744073709551615 + +-- !select -- +\N + +-- !select -- +null + +-- !select -- +true + +-- !select -- +false + +-- !select -- +100 + +-- !select -- +10000 + +-- !select -- +1000000000 + +-- !select -- +1152921504606846976 + +-- !select -- +6.18 + +-- !select -- +"abcd" + +-- !select -- +{} + +-- !select -- +{"k1":"v31","k2":300} + +-- !select -- +[] + +-- !select -- +[123,456] + +-- !select -- +["abc","def"] + +-- !select -- +[null,true,false,100,6.18,"abc"] + +-- !select -- +[{"k1":"v41","k2":400},1,"a",3.14] + +-- !select -- +{"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +1 \N +2 1 +3 1 +4 1 +5 1 +6 1 +7 1 +8 1 +9 1 +10 1 +11 1 +12 1 +13 1 +14 1 +15 1 +16 1 +17 1 +18 1 +26 \N +27 1 +28 1 +29 1 +30 1 +31 1 + +-- !select -- +1 + +-- !select -- +0 + +-- !select -- +\N + +-- !select -- +1 \N \N +2 null null +3 true null +4 false null +5 100 null +6 10000 null +7 1000000000 null +8 1152921504606846976 null +9 6.18 null +10 "abcd" null +11 {} null +12 {"k1":"v31","k2":300} "v31" +13 [] null +14 [123,456] null +15 ["abc","def"] null +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] ["v41"] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "v31" +26 \N \N +27 {"k1":"v1","k2":200} "v1" +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} null +29 12524337771678448270 null +30 -9223372036854775808 null +31 18446744073709551615 null + +-- !select -- +1 \N \N +2 null [null,null] +3 true [null,null] +4 false [null,null] +5 100 [null,null] +6 10000 [null,null] +7 1000000000 [null,null] +8 1152921504606846976 [null,null] +9 6.18 [null,null] +10 "abcd" [null,null] +11 {} [null,null] +12 {"k1":"v31","k2":300} [300,null] +13 [] [null,null] +14 [123,456] [null,456] +15 ["abc","def"] [null,"def"] +16 [null,true,false,100,6.18,"abc"] [null,true] +17 [{"k1":"v41","k2":400},1,"a",3.14] [[400],1] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [300,null] +26 \N \N +27 {"k1":"v1","k2":200} [200,null] +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} [null,null] +29 12524337771678448270 [null,null] +30 -9223372036854775808 [null,null] +31 18446744073709551615 [null,null] + +-- !select -- +1 \N \N +2 null [null,null] +3 true [null,null] +4 false [null,null] +5 100 [null,null] +6 10000 [null,null] +7 1000000000 [null,null] +8 1152921504606846976 [null,null] +9 6.18 [null,null] +10 "abcd" [null,null] +11 {} [null,null] +12 {"k1":"v31","k2":300} [300,null] +13 [] [null,null] +14 [123,456] [null,null] +15 ["abc","def"] [null,null] +16 [null,true,false,100,6.18,"abc"] [null,null] +17 [{"k1":"v41","k2":400},1,"a",3.14] [[400],null] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [300,null] +26 \N \N +27 {"k1":"v1","k2":200} [200,null] +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} [null,null] +29 12524337771678448270 [null,null] +30 -9223372036854775808 [null,null] +31 18446744073709551615 [null,null] + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select -- +1 \N \N +2 null [null,null,null] +3 true [null,null,null] +4 false [null,null,null] +5 100 [null,null,null] +6 10000 [null,null,null] +7 1000000000 [null,null,null] +8 1152921504606846976 [null,null,null] +9 6.18 [null,null,null] +10 "abcd" [null,null,null] +11 {} [null,null,null] +12 {"k1":"v31","k2":300} [null,null,null] +13 [] [null,null,null] +14 [123,456] [null,null,null] +15 ["abc","def"] [null,null,null] +16 [null,true,false,100,6.18,"abc"] [null,null,null] +17 [{"k1":"v41","k2":400},1,"a",3.14] [null,null,null] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} ["v41",400,"a"] +26 \N \N +27 {"k1":"v1","k2":200} [null,null,null] +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} [null,null,null] +29 12524337771678448270 [null,null,null] +30 -9223372036854775808 [null,null,null] +31 18446744073709551615 [null,null,null] + diff --git a/regression-test/data/nereids_p0/jsonb_p0/test_jsonb_load_unique_key_and_function.out b/regression-test/data/nereids_p0/jsonb_p0/test_jsonb_load_unique_key_and_function.out new file mode 100644 index 00000000000000..632badf32e1de8 --- /dev/null +++ b/regression-test/data/nereids_p0/jsonb_p0/test_jsonb_load_unique_key_and_function.out @@ -0,0 +1,5869 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select -- +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31","k2":300} +13 [] +14 [123,456] +15 ["abc","def"] +16 [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} + +-- !select -- +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31","k2":300} +13 [] +14 [123,456] +15 ["abc","def"] +16 [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N +27 {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} + +-- !jsonb_extract_select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" "abcd" +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} "v31" +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "v31" +26 \N \N +27 {"k1":"v1","k2":200} "v1" +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"k1.a1":"v31","k2":300} + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} "v31" + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] \N +14 [123,456] 123 +15 ["abc","def"] "abc" +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] {"k1":"v41","k2":400} +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] "def" +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] "a" +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] 3.14 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 6.18 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] "abc" +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [{"k1":"v41","k2":400},1,"a",3.14] +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v41","k2":400} +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "a" +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} "a" +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v41","k2":400} +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !jsonb_extract_multipath -- +1 \N \N +2 null [null,null,null,null] +3 true [true,null,null,null] +4 false [false,null,null,null] +5 100 [100,null,null,null] +6 10000 [10000,null,null,null] +7 1000000000 [1000000000,null,null,null] +8 1152921504606846976 [1152921504606846976,null,null,null] +9 6.18 [6.18,null,null,null] +10 "abcd" ["abcd",null,null,null] +11 {} [{},{},null,{}] +12 {"k1":"v31","k2":300} [{"k1":"v31","k2":300},{"k1":"v31","k2":300},"v31",{"k1":"v31","k2":300}] +13 [] [[],null,null,null] +14 [123,456] [[123,456],null,null,123] +15 ["abc","def"] [["abc","def"],null,null,"abc"] +16 [null,true,false,100,6.18,"abc"] [[null,true,false,100,6.18,"abc"],null,null,null] +17 [{"k1":"v41","k2":400},1,"a",3.14] [[{"k1":"v41","k2":400},1,"a",3.14],null,null,{"k1":"v41","k2":400}] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [{"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]},{"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]},"v31",{"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]}] +26 \N \N +27 {"k1":"v1","k2":200} [{"k1":"v1","k2":200},{"k1":"v1","k2":200},"v1",{"k1":"v1","k2":200}] +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} [{"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"},{"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"},null,{"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"}] + +-- !jsonb_extract_string_select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" abcd +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} v31 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} v31 +26 \N \N +27 {"k1":"v1","k2":200} v1 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] \N +14 [123,456] 123 +15 ["abc","def"] abc +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] {"k1":"v41","k2":400} +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] def +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] a +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] 3.14 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 6.18 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] abc +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} [{"k1":"v41","k2":400},1,"a",3.14] +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v41","k2":400} +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} a +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} a +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v41","k2":400} +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !jsonb_extract_int_select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !jsonb_extract_bigint_select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300 +26 \N \N +27 {"k1":"v1","k2":200} 200 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !jsonb_extract_double_select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100.0 +6 10000 10000.0 +7 1000000000 1.0E9 +8 1152921504606846976 1.15292150460684698E18 +9 6.18 6.18 +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} 300.0 +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 300.0 +26 \N \N +27 {"k1":"v1","k2":200} 200.0 +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 123.0 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] 456.0 +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] 1.0 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 100.0 +17 [{"k1":"v41","k2":400},1,"a",3.14] 3.14 +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] 6.18 +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1.0 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 1.0 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} 3.14 +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !jsonb_extract_bool_select -- +1 \N \N +2 null \N +3 true true +4 false false +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !jsonb_extract_isnull_select -- +1 \N \N +2 null true +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} false +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} false +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} false +12 {"k1":"v31","k2":300} false +13 [] \N +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !jsonb_exists_path_select -- +1 \N \N +2 null true +3 true true +4 false true +5 100 true +6 10000 true +7 1000000000 true +8 1152921504606846976 true +9 6.18 true +10 "abcd" true +11 {} true +12 {"k1":"v31","k2":300} true +13 [] true +14 [123,456] true +15 ["abc","def"] true +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} true + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} true +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} true +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} true +12 {"k1":"v31","k2":300} true +13 [] false +14 [123,456] true +15 ["abc","def"] true +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} true +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} true + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] true +15 ["abc","def"] true +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] true +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] true +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} true +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !select -- +1 \N \N +2 null false +3 true false +4 false false +5 100 false +6 10000 false +7 1000000000 false +8 1152921504606846976 false +9 6.18 false +10 "abcd" false +11 {} false +12 {"k1":"v31","k2":300} false +13 [] false +14 [123,456] false +15 ["abc","def"] false +16 [null,true,false,100,6.18,"abc"] false +17 [{"k1":"v41","k2":400},1,"a",3.14] false +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} false +26 \N \N +27 {"k1":"v1","k2":200} false +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} false + +-- !jsonb_type_select -- +1 \N \N +2 null null +3 true bool +4 false bool +5 100 int +6 10000 int +7 1000000000 int +8 1152921504606846976 bigint +9 6.18 double +10 "abcd" string +11 {} object +12 {"k1":"v31","k2":300} object +13 [] array +14 [123,456] array +15 ["abc","def"] array +16 [null,true,false,100,6.18,"abc"] array +17 [{"k1":"v41","k2":400},1,"a",3.14] array +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} object +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} object + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} string +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} string +26 \N \N +27 {"k1":"v1","k2":200} string +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} int +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} int +26 \N \N +27 {"k1":"v1","k2":200} int +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} object +12 {"k1":"v31","k2":300} object +13 [] \N +14 [123,456] int +15 ["abc","def"] string +16 [null,true,false,100,6.18,"abc"] null +17 [{"k1":"v41","k2":400},1,"a",3.14] object +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} object +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} object + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] int +15 ["abc","def"] string +16 [null,true,false,100,6.18,"abc"] bool +17 [{"k1":"v41","k2":400},1,"a",3.14] int +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] bool +17 [{"k1":"v41","k2":400},1,"a",3.14] string +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] int +17 [{"k1":"v41","k2":400},1,"a",3.14] double +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] double +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] string +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} array +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} int +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} string +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} double +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} double +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} double +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} string +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} int +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} object +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} double +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !cast_from_select -- +1 \N \N +2 null \N +3 true true +4 false false +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100.0 +6 10000 10000.0 +7 1000000000 1.0E9 +8 1152921504606846976 1.15292150460684698E18 +9 6.18 6.18 +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" "abcd" +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} + +-- !cast_to_select -- +1 \N \N +2 null \N +3 true true +4 false false +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1.15292150460685e+18 +9 6.18 6.18 +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N + +-- !select -- +1 \N \N +2 null null +3 true true +4 false false +5 100 100 +6 10000 10000 +7 1000000000 1000000000 +8 1152921504606846976 1152921504606846976 +9 6.18 6.18 +10 "abcd" "abcd" +11 {} {} +12 {"k1":"v31","k2":300} {"k1":"v31","k2":300} +13 [] [] +14 [123,456] [123,456] +15 ["abc","def"] ["abc","def"] +16 [null,true,false,100,6.18,"abc"] [null,true,false,100,6.18,"abc"] +17 [{"k1":"v41","k2":400},1,"a",3.14] [{"k1":"v41","k2":400},1,"a",3.14] +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} +26 \N \N +27 {"k1":"v1","k2":200} {"k1":"v1","k2":200} +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} + +-- !select -- +\N + +-- !select -- +null + +-- !select -- +true + +-- !select -- +false + +-- !select -- +100 + +-- !select -- +10000 + +-- !select -- +1000000000 + +-- !select -- +1152921504606846976 + +-- !select -- +6.18 + +-- !select -- +"abcd" + +-- !select -- +{} + +-- !select -- +{"k1":"v31","k2":300} + +-- !select -- +[] + +-- !select -- +[123,456] + +-- !select -- +["abc","def"] + +-- !select -- +[null,true,false,100,6.18,"abc"] + +-- !select -- +[{"k1":"v41","k2":400},1,"a",3.14] + +-- !select -- +{"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + +-- !select -- +\N + diff --git a/regression-test/data/nereids_p0/jsonb_p0/test_jsonb_unique_key.csv b/regression-test/data/nereids_p0/jsonb_p0/test_jsonb_unique_key.csv new file mode 100644 index 00000000000000..88cc69278673c5 --- /dev/null +++ b/regression-test/data/nereids_p0/jsonb_p0/test_jsonb_unique_key.csv @@ -0,0 +1,75 @@ +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31", "k2": 300} +13 [] +14 [123, 456] +15 ["abc", "def"] +16 [null, true, false, 100, 6.18, "abc"] +17 [{"k1":"v41", "k2": 400}, 1, "a", 3.14] +18 {"k1":"v31", "k2": 300, "a1": [{"k1":"v41", "k2": 400}, 1, "a", 3.14]} +19 '' +20 'abc' +21 abc +22 100x +23 6.a8 +24 {x +25 [123, abc] +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31", "k2": 300} +13 [] +14 [123, 456] +15 ["abc", "def"] +16 [null, true, false, 100, 6.18, "abc"] +17 [{"k1":"v41", "k2": 400}, 1, "a", 3.14] +18 {"k1":"v31", "k2": 300, "a1": [{"k1":"v41", "k2": 400}, 1, "a", 3.14]} +19 '' +20 'abc' +21 abc +22 100x +23 6.a8 +24 {x +25 [123, abc] +1 \N +2 null +3 true +4 false +5 100 +6 10000 +7 1000000000 +8 1152921504606846976 +9 6.18 +10 "abcd" +11 {} +12 {"k1":"v31", "k2": 300} +13 [] +14 [123, 456] +15 ["abc", "def"] +16 [null, true, false, 100, 6.18, "abc"] +17 [{"k1":"v41", "k2": 400}, 1, "a", 3.14] +18 {"k1":"v31", "k2": 300, "a1": [{"k1":"v41", "k2": 400}, 1, "a", 3.14]} +19 '' +20 'abc' +21 abc +22 100x +23 6.a8 +24 {x +25 [123, abc] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query18.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query18.out index 44c121660a3fe1..9ab6b66e7073d6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query18.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query18.out @@ -10,15 +10,15 @@ PhysicalResultSink --------------hashAgg[LOCAL] ----------------PhysicalRepeat ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](customer.c_current_cdemo_sk = cd2.cd_demo_sk) +--------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +----------------------PhysicalProject +------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer_demographics] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +--------------------------hashJoin[INNER_JOIN](customer.c_current_cdemo_sk = cd2.cd_demo_sk) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_demographics] ----------------------------PhysicalDistribute ------------------------------PhysicalProject --------------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query59.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query59.out index 74f0058cc1fb92..d5d282966a7bf5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query59.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query59.out @@ -17,33 +17,36 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------PhysicalDistribute --------PhysicalTopN ----------PhysicalProject -------------hashJoin[INNER_JOIN](y.s_store_id1 = x.s_store_id2)(wss.ss_store_sk = store.s_store_sk) ---------------hashJoin[INNER_JOIN](expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 52)) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk) -----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq1) +------------hashJoin[INNER_JOIN](y.s_store_id1 = x.s_store_id2)(expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 52)) +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk) +--------------------PhysicalDistribute +----------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq1) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((d.d_month_seq <= 1207)(d.d_month_seq >= 1196)) +------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalOlapScan[store] +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq2) --------------------------PhysicalDistribute ----------------------------PhysicalProject ------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((d.d_month_seq <= 1207)(d.d_month_seq >= 1196)) +------------------------------filter((d.d_month_seq <= 1219)(d.d_month_seq >= 1208)) --------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store] -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq2) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((d.d_month_seq <= 1219)(d.d_month_seq >= 1208)) ---------------------------PhysicalOlapScan[date_dim] ---------------PhysicalDistribute -----------------PhysicalProject -------------------PhysicalOlapScan[store] +------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query61.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query61.out index 9dcfea291d8c4c..69f00c102e4291 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query61.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query61.out @@ -27,9 +27,9 @@ PhysicalResultSink ------------------------------------PhysicalOlapScan[customer] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) ---------------------------------------PhysicalDistribute -----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) ------------------------------------------PhysicalProject --------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ----------------------------------------------PhysicalProject @@ -40,12 +40,12 @@ PhysicalResultSink ----------------------------------------------------PhysicalOlapScan[store] ------------------------------------------PhysicalDistribute --------------------------------------------PhysicalProject -----------------------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 1999)) -------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------filter((((cast(p_channel_dmail as VARCHAR(*)) = 'Y') OR (cast(p_channel_email as VARCHAR(*)) = 'Y')) OR (cast(p_channel_tv as VARCHAR(*)) = 'Y'))) +------------------------------------------------PhysicalOlapScan[promotion] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((((cast(p_channel_dmail as VARCHAR(*)) = 'Y') OR (cast(p_channel_email as VARCHAR(*)) = 'Y')) OR (cast(p_channel_tv as VARCHAR(*)) = 'Y'))) ---------------------------------------------PhysicalOlapScan[promotion] +------------------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 1999)) +--------------------------------------------PhysicalOlapScan[date_dim] ----------PhysicalDistribute ------------hashAgg[GLOBAL] --------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out index d52d1ff6ae7b57..512604bb57e66e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out @@ -10,40 +10,41 @@ PhysicalResultSink --------------PhysicalProject ----------------hashJoin[INNER_JOIN](tmp.time_sk = time_dim.t_time_sk) ------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](tmp.sold_item_sk = item.i_item_sk) -----------------------PhysicalDistribute -------------------------PhysicalUnion ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] ---------------------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](tmp.sold_item_sk = item.i_item_sk) +------------------------PhysicalDistribute +--------------------------PhysicalUnion +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) ---------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] ---------------------------------PhysicalDistribute +------------------------------------PhysicalOlapScan[web_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) +----------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) ---------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] ---------------------------------PhysicalDistribute +------------------------------------PhysicalOlapScan[catalog_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) +----------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) ---------------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((item.i_manager_id = 1)) -----------------------------PhysicalOlapScan[item] +------------------------------------PhysicalOlapScan[store_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) +----------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((item.i_manager_id = 1)) +------------------------------PhysicalOlapScan[item] ------------------PhysicalDistribute --------------------PhysicalProject ----------------------filter(((cast(t_meal_time as VARCHAR(*)) = 'breakfast') OR (cast(t_meal_time as VARCHAR(*)) = 'dinner'))) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query72.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query72.out index 888557d30d4bb1..fdae8fece66bdf 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query72.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query72.out @@ -13,49 +13,51 @@ PhysicalResultSink --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN](warehouse.w_warehouse_sk = inventory.inv_warehouse_sk) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](inventory.inv_date_sk = d2.d_date_sk)(d1.d_week_seq = d2.d_week_seq) -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = inventory.inv_item_sk)(inventory.inv_quantity_on_hand < catalog_sales.cs_quantity) ---------------------------------PhysicalOlapScan[inventory] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[LEFT_OUTER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) -------------------------------------------PhysicalDistribute ---------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_date_sk = d3.d_date_sk)(d3.d_date > cast((cast(d_date as BIGINT) + 5) as DATEV2)) -------------------------------------------------PhysicalDistribute ---------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk) -----------------------------------------------------PhysicalDistribute -------------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d1.d_date_sk) ---------------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_hdemo_sk = household_demographics.hd_demo_sk) -----------------------------------------------------------PhysicalProject -------------------------------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------------------------------PhysicalDistribute -------------------------------------------------------------PhysicalProject ---------------------------------------------------------------filter((cast(hd_buy_potential as VARCHAR(*)) = '501-1000')) -----------------------------------------------------------------PhysicalOlapScan[household_demographics] ---------------------------------------------------------PhysicalDistribute -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((d1.d_year = 2002)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------------PhysicalDistribute -------------------------------------------------------PhysicalProject ---------------------------------------------------------filter((cast(cd_marital_status as VARCHAR(*)) = 'W')) -----------------------------------------------------------PhysicalOlapScan[customer_demographics] -------------------------------------------------PhysicalDistribute ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalDistribute ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[promotion] +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = inventory.inv_item_sk)(inventory.inv_date_sk = d2.d_date_sk)(inventory.inv_quantity_on_hand < catalog_sales.cs_quantity) +----------------------------PhysicalDistribute +------------------------------PhysicalOlapScan[inventory] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[date_dim] +--------------------------------hashJoin[INNER_JOIN](d1.d_week_seq = d2.d_week_seq) +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------hashJoin[LEFT_OUTER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_date_sk = d3.d_date_sk)(d3.d_date > cast((cast(d_date as BIGINT) + 5) as DATEV2)) +--------------------------------------------------PhysicalDistribute +----------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk) +------------------------------------------------------PhysicalDistribute +--------------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d1.d_date_sk) +----------------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_hdemo_sk = household_demographics.hd_demo_sk) +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------------------------------PhysicalDistribute +--------------------------------------------------------------PhysicalProject +----------------------------------------------------------------filter((cast(hd_buy_potential as VARCHAR(*)) = '501-1000')) +------------------------------------------------------------------PhysicalOlapScan[household_demographics] +----------------------------------------------------------PhysicalDistribute +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((d1.d_year = 2002)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------PhysicalDistribute +--------------------------------------------------------PhysicalProject +----------------------------------------------------------filter((cast(cd_marital_status as VARCHAR(*)) = 'W')) +------------------------------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------------------------PhysicalDistribute +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[item] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[promotion] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject ----------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out index ea982e8c2933a2..bfcec6ce4127f4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out @@ -25,11 +25,11 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalTopN ----------PhysicalProject ------------hashJoin[INNER_JOIN](ctr1.ctr_state = ctr2.ctr_state)(cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE)) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) -------------------PhysicalDistribute ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------PhysicalDistribute +--------------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) +----------------PhysicalDistribute +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------PhysicalDistribute +------------------PhysicalProject --------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) ----------------------PhysicalProject ------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out index 4f20f7bf82aa1b..919fdb60190fba 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out @@ -8,10 +8,10 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) +----------------hashJoin[INNER_JOIN](catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk) ------------------PhysicalProject --------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk) -----------------------hashJoin[INNER_JOIN](catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk) +----------------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) ------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_date_sk = date_dim.d_date_sk) --------------------------PhysicalProject ----------------------------PhysicalOlapScan[catalog_sales] @@ -21,11 +21,11 @@ PhysicalResultSink --------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------PhysicalOlapScan[call_center] +----------------------------PhysicalOlapScan[warehouse] ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------PhysicalOlapScan[ship_mode] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------PhysicalOlapScan[warehouse] +----------------------PhysicalOlapScan[call_center] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q8.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q8.out index 07a9cd9ca20154..e699fc97c2bd55 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q8.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q8.out @@ -16,33 +16,36 @@ PhysicalResultSink --------------------------PhysicalOlapScan[supplier] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](n1.n_regionkey = region.r_regionkey) -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](customer.c_nationkey = n1.n_nationkey) -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](orders.o_custkey = customer.c_custkey) +----------------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](orders.o_custkey = customer.c_custkey) +------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer] ---------------------------------------PhysicalDistribute -----------------------------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) -------------------------------------------PhysicalProject ---------------------------------------------filter((orders.o_orderdate <= 1996-12-31)(orders.o_orderdate >= 1995-01-01)) -----------------------------------------------PhysicalOlapScan[orders] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) +----------------------------------------filter((orders.o_orderdate <= 1996-12-31)(orders.o_orderdate >= 1995-01-01)) +------------------------------------------PhysicalOlapScan[orders] +------------------------------------PhysicalDistribute +--------------------------------------hashJoin[INNER_JOIN](customer.c_nationkey = n1.n_nationkey) +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[customer] +----------------------------------------PhysicalDistribute +------------------------------------------hashJoin[INNER_JOIN](n1.n_regionkey = region.r_regionkey) +--------------------------------------------PhysicalDistribute ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[lineitem] -----------------------------------------------PhysicalDistribute -------------------------------------------------PhysicalProject ---------------------------------------------------filter((part.p_type = 'ECONOMY ANODIZED STEEL')) -----------------------------------------------------PhysicalOlapScan[part] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[nation] +------------------------------------------------PhysicalOlapScan[nation] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------filter((region.r_name = 'AMERICA')) +--------------------------------------------------PhysicalOlapScan[region] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((region.r_name = 'AMERICA')) -------------------------------------PhysicalOlapScan[region] +----------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[lineitem] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((part.p_type = 'ECONOMY ANODIZED STEEL')) +------------------------------------------PhysicalOlapScan[part] --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/performance_p0/redundant_conjuncts.out b/regression-test/data/performance_p0/redundant_conjuncts.out index f82e0c94536911..6a7fd4fd9318ac 100644 --- a/regression-test/data/performance_p0/redundant_conjuncts.out +++ b/regression-test/data/performance_p0/redundant_conjuncts.out @@ -6,6 +6,7 @@ PLAN FRAGMENT 0 PARTITION: HASH_PARTITIONED: `default_cluster:regression_test_performance_p0`.`redundant_conjuncts`.`k1` VRESULT SINK + MYSQL_PROTOCAL 0:VOlapScanNode TABLE: default_cluster:regression_test_performance_p0.redundant_conjuncts(redundant_conjuncts), PREAGGREGATION: OFF. Reason: No AggregateInfo @@ -21,6 +22,7 @@ PLAN FRAGMENT 0 PARTITION: HASH_PARTITIONED: `default_cluster:regression_test_performance_p0`.`redundant_conjuncts`.`k1` VRESULT SINK + MYSQL_PROTOCAL 0:VOlapScanNode TABLE: default_cluster:regression_test_performance_p0.redundant_conjuncts(redundant_conjuncts), PREAGGREGATION: OFF. Reason: No AggregateInfo diff --git a/regression-test/data/query_p0/sql_functions/array_functions/test_array_functions.out b/regression-test/data/query_p0/sql_functions/array_functions/test_array_functions.out index 086a172e3e3d09..fb3ef0e74758f2 100644 --- a/regression-test/data/query_p0/sql_functions/array_functions/test_array_functions.out +++ b/regression-test/data/query_p0/sql_functions/array_functions/test_array_functions.out @@ -209,19 +209,19 @@ 9 \N -- !select -- -1 [1, 2, 3] +1 [3, 2, 1] 2 [4, 5] 3 \N -4 [1, 2, 3, 4, 5] +4 [3, 2, 1, 4, 5] 5 \N 6 \N 7 \N -8 [1, 2, 3, 4, NULL] -9 [1, 2, 3] +8 [NULL, 3, 2, 1, 4] +9 [3, 2, 1] -- !select -- -1 [2023-02-05, 2023-02-06, 2023-02-07] -2 [2023-01-05, 2023-01-06, 2023-01-07] +1 [2023-02-07, 2023-02-06, 2023-02-05] +2 [2023-01-06, 2023-01-07, 2023-01-05] 3 \N 4 \N 5 \N @@ -231,8 +231,8 @@ 9 \N -- !select -- -1 [2022-10-15 10:30:00.999, 2022-08-31 12:00:00.999, 2022-10-16 10:30:00.999] -2 [2022-11-15 10:30:00.999, 2022-01-31 12:00:00.999, 2022-11-16 10:30:00.999] +1 [2022-10-16 10:30:00.999, 2022-10-15 10:30:00.999, 2022-08-31 12:00:00.999] +2 [2022-11-15 10:30:00.999, 2022-11-16 10:30:00.999, 2022-01-31 12:00:00.999] 3 \N 4 \N 5 \N @@ -243,7 +243,7 @@ -- !select -- 1 [111.111, 222.222, 333.333] -2 [333.333, 444.444, 555.556] +2 [444.444, 555.556, 333.333] 3 \N 4 \N 5 \N @@ -252,6 +252,17 @@ 8 \N 9 \N +-- !select -- +1 [3, 2, 222, 1] +2 \N +3 \N +4 [NULL, 3, 23, 2, 1, 4, 5] +5 \N +6 \N +7 \N +8 [NULL, 8, 10, 3, 9, 2, 1, 4] +9 [NULL, 3, 2, 1, 12] + -- !select -- 1 [3] 2 [4] @@ -1645,8 +1656,8 @@ [1, 2, 3] 1,2,3 -- !select_union -- -10005 [10005, NULL, NULL] [10005, NULL, 1, 2, 3] -10006 [60002, 60002, 60003, NULL, 60005] [60002, 60003, NULL, 60005, 1, 2, 3] +10005 [10005, NULL, NULL] [NULL, 3, 10005, 2, 1] +10006 [60002, 60002, 60003, NULL, 60005] [NULL, 3, 60002, 60005, 60003, 2, 1] -- !select_except -- 10005 [10005, NULL, NULL] [10005, NULL] diff --git a/regression-test/data/query_p0/sql_functions/array_functions/test_array_functions_by_literal.out b/regression-test/data/query_p0/sql_functions/array_functions/test_array_functions_by_literal.out index 5d7965b29db5e9..014067c7ec21cd 100644 --- a/regression-test/data/query_p0/sql_functions/array_functions/test_array_functions_by_literal.out +++ b/regression-test/data/query_p0/sql_functions/array_functions/test_array_functions_by_literal.out @@ -39,7 +39,7 @@ [] -- !sql_11 -- -[1.0, 2.0, NULL] +[NULL, 1.0, 2.0] -- !sql_14 -- [2.0, NULL, NULL, 2.0] @@ -459,7 +459,7 @@ false true -- !sql -- -[1, 2, 3, 4] +[3, 2, 1, 4] -- !sql -- [1] @@ -468,7 +468,7 @@ true [3, 2] -- !sql -- -[1, 2, 3, 4, NULL] +[NULL, 3, 2, 1, 4] -- !sql -- [1] @@ -477,7 +477,7 @@ true [3, 2] -- !sql -- -[1, 0] +[0, 1] -- !sql -- [0] @@ -495,7 +495,7 @@ true [] -- !sql -- -[1, 2, 3] +[3, 2, 1] -- !sql -- [] @@ -504,7 +504,7 @@ true [] -- !sql -- -[NULL, 1, 2, 3] +[NULL, 3, 2, 1] -- !sql -- [NULL] @@ -513,7 +513,7 @@ true [] -- !sql -- -[1, 100000000] +[100000000, 1] -- !sql -- [1] @@ -531,7 +531,7 @@ true [2023-02-04 23:07:34.999] -- !sql -- -[2023-02-06, 2023-02-05, 2023-02-07] +[2023-02-07, 2023-02-06, 2023-02-05] -- !sql -- [2023-02-06] @@ -596,6 +596,18 @@ true -- !sql_intersect_4 -- [] +-- !sql_union_1 -- +[NULL, 3, 2, 1] + +-- !sql_union_2 -- +[NULL, 3, 2, 1] + +-- !sql_union_3 -- +[NULL, 3, 2, 1] + +-- !sql_union_4 -- +[NULL, 3, 2, 1] + -- !sql -- [2, 3, 4, 5, 6] @@ -755,18 +767,6 @@ _ -- !sql -- [2023-03-05 12:23:24.999, 2023-03-05 15:23:23.997] --- !select_array_shuffle1 -- -17 [1, 3, 2, NULL, 3, 4, NULL, 4] [1, 3, 2, NULL, 3, 4, NULL, 4] - --- !select_array_shuffle2 -- -6.666 [3.333, 2.222, 1.111] [3.333, 2.222, 1.111] - --- !select_array_shuffle3 -- -4 ["aaa", "bbb", NULL, "fff"] ["aaa", "bbb", NULL, "fff"] - --- !select_array_shuffle4 -- -4 ["2020-01-02", "2021-01-01", "2022-01-03", "1996-04-17"] ["2020-01-02", "2021-01-01", "2022-01-03", "1996-04-17"] - -- !sql -- [{"a", "d"}, {"b", "e"}, {"c", "f"}] diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy index 9beb8e6213a4c9..c3f30d7ca28a83 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy @@ -353,7 +353,8 @@ class Suite implements GroovyInterceptable { } String uploadToHdfs(String localFile) { - String dataDir = context.config.dataPath + "/" + group + "/" + // as group can be rewrite the origin data file not relate to group + String dataDir = context.config.dataPath + "/" localFile = dataDir + localFile String hdfsFs = context.config.otherConfigs.get("hdfsFs") String hdfsUser = context.config.otherConfigs.get("hdfsUser") diff --git a/regression-test/suites/datatype_p0/nested_types/load.groovy b/regression-test/suites/datatype_p0/nested_types/load.groovy new file mode 100644 index 00000000000000..564628ac5c63f5 --- /dev/null +++ b/regression-test/suites/datatype_p0/nested_types/load.groovy @@ -0,0 +1,532 @@ +// 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. + +suite("load") { + // ddl begin + sql """ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false')""" + sql """set enable_nereids_planner=false""" + def dataFile = """test_scalar_types_100.csv""" + + // define dup key table1 with scala types + def scala_table_dup = "tbl_scalar_types_dup" + sql "DROP TABLE IF EXISTS ${scala_table_dup}" + sql """ + CREATE TABLE IF NOT EXISTS ${scala_table_dup} ( + `k1` bigint(11) NULL, + `c_bool` boolean NULL, + `c_tinyint` tinyint(4) NULL, + `c_smallint` smallint(6) NULL, + `c_int` int(11) NULL, + `c_bigint` bigint(20) NULL, + `c_largeint` largeint(40) NULL, + `c_float` float NULL, + `c_double` double NULL, + `c_decimal` decimal(20, 3) NULL, + `c_decimalv3` decimalv3(20, 3) NULL, + `c_date` date NULL, + `c_datetime` datetime NULL, + `c_datev2` datev2 NULL, + `c_datetimev2` datetimev2(0) NULL, + `c_char` char(15) NULL, + `c_varchar` varchar(100) NULL, + `c_string` text NULL + ) ENGINE=OLAP + DUPLICATE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + // load data + streamLoad { + table scala_table_dup + file dataFile + time 60000 + + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + assertEquals(100, json.NumberTotalRows) + assertEquals(100, json.NumberLoadedRows) + } + } + + // insert two NULL rows + sql """INSERT INTO ${scala_table_dup} VALUES (NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, + NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)""" + sql """INSERT INTO ${scala_table_dup} VALUES (NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, + NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)""" + + + // define dup key table with nested table types with one nested scala + def nested_table_dup = "tbl_array_nested_types_dup" + sql "DROP TABLE IF EXISTS ${nested_table_dup}" + sql """ + CREATE TABLE IF NOT EXISTS ${nested_table_dup} ( + `k1` bigint(11) NULL, + `c_bool` array NULL, + `c_tinyint` array NULL, + `c_smallint` array NULL, + `c_int` array NULL, + `c_bigint` array NULL, + `c_largeint` array NULL, + `c_float` array NULL, + `c_double` array NULL, + `c_decimal` array NULL, + `c_decimalv3` array NULL, + `c_date` array NULL, + `c_datetime` array NULL, + `c_datev2` array NULL, + `c_datetimev2` array NULL, + `c_char` array NULL, + `c_varchar` array NULL, + `c_string` array NULL + ) ENGINE=OLAP + DUPLICATE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + // define dup key table with nested table types with two nested scala + def nested_table_dup2 = "tbl_array_nested_types_dup2" + sql "DROP TABLE IF EXISTS ${nested_table_dup2}" + sql """ + CREATE TABLE IF NOT EXISTS ${nested_table_dup2} ( + `k1` bigint(11) NULL, + `c_bool` array> NULL, + `c_tinyint` array> NULL, + `c_smallint` array> NULL, + `c_int` array> NULL, + `c_bigint` array> NULL, + `c_largeint` array> NULL, + `c_float` array> NULL, + `c_double` array> NULL, + `c_decimal` array> NULL, + `c_decimalv3` array> NULL, + `c_date` array> NULL, + `c_datetime` array> NULL, + `c_datev2` array> NULL, + `c_datetimev2` array> NULL, + `c_char` array> NULL, + `c_varchar` array> NULL, + `c_string` array> NULL + ) ENGINE=OLAP + DUPLICATE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + // define dup key table with map types with one nested scala + def nested_table_map_dup = "tbl_map_types_dup" + sql "DROP TABLE IF EXISTS ${nested_table_map_dup}" + sql """ + CREATE TABLE IF NOT EXISTS ${nested_table_map_dup} ( + `k1` bigint(11) NULL, + `c_bool` map NULL, + `c_tinyint` map NULL, + `c_smallint` map NULL, + `c_int` map NULL, + `c_bigint` map NULL, + `c_largeint` map NULL, + `c_float` map NULL, + `c_double` map NULL, + `c_decimal` map NULL, + `c_decimalv3` map NULL, + `c_date` map NULL, + `c_datetime` map NULL, + `c_datev2` map NULL, + `c_datetimev2` map NULL, + `c_char` map NULL, + `c_varchar` map NULL, + `c_string` map NULL + ) ENGINE=OLAP + DUPLICATE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + + // define dup key table with array nested map table types with one nested scala + def nested_table_array_map_dup = "tbl_array_map_types_dup" + sql "DROP TABLE IF EXISTS ${nested_table_array_map_dup}" + sql """ + CREATE TABLE IF NOT EXISTS ${nested_table_array_map_dup} ( + `k1` bigint(11) NULL, + `c_bool` array> NULL, + `c_tinyint` array> NULL, + `c_smallint` array> NULL, + `c_int` array> NULL, + `c_bigint` array> NULL, + `c_largeint` array> NULL, + `c_float` array> NULL, + `c_double` array> NULL, + `c_decimal` array> NULL, + `c_decimalv3` array> NULL, + `c_date` array> NULL, + `c_datetime` array> NULL, + `c_datev2` array> NULL, + `c_datetimev2` array> NULL, + `c_char` array> NULL, + `c_varchar` array> NULL, + `c_string` array> NULL + ) ENGINE=OLAP + DUPLICATE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + // define dup key table with map nested value array table types with one nested scala + def nested_table_map_array_dup = "tbl_map_array_types_dup" + sql "DROP TABLE IF EXISTS ${nested_table_map_array_dup}" + sql """ + CREATE TABLE IF NOT EXISTS ${nested_table_map_array_dup} ( + `k1` bigint(11) NULL, + `c_bool` map> NULL, + `c_tinyint` map> NULL, + `c_smallint` map> NULL, + `c_int` map> NULL, + `c_bigint` map> NULL, + `c_largeint` map> NULL, + `c_float` map> NULL, + `c_double` map> NULL, + `c_decimal` map> NULL, + `c_decimalv3` map> NULL, + `c_date` map> NULL, + `c_datetime` map> NULL, + `c_datev2` map> NULL, + `c_datetimev2` map> NULL, + `c_char` map> NULL, + `c_varchar` map> NULL, + `c_string` map> NULL + ) ENGINE=OLAP + DUPLICATE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + // unique table + // define mor key table with nested table types with one nested scala + def nested_table_mor = "tbl_array_nested_types_mor" + sql "DROP TABLE IF EXISTS ${nested_table_mor}" + sql """ + CREATE TABLE IF NOT EXISTS ${nested_table_mor} ( + `k1` bigint(11) NULL, + `c_bool` array NULL, + `c_tinyint` array NULL, + `c_smallint` array NULL, + `c_int` array NULL, + `c_bigint` array NULL, + `c_largeint` array NULL, + `c_float` array NULL, + `c_double` array NULL, + `c_decimal` array NULL, + `c_decimalv3` array NULL, + `c_date` array NULL, + `c_datetime` array NULL, + `c_datev2` array NULL, + `c_datetimev2` array NULL, + `c_char` array NULL, + `c_varchar` array NULL, + `c_string` array NULL + ) ENGINE=OLAP + UNIQUE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + // define mor key table with nested table types with two nested scala + def nested_table_mor2 = "tbl_array_nested_types_mor2" + sql "DROP TABLE IF EXISTS ${nested_table_mor2}" + sql """ + CREATE TABLE IF NOT EXISTS ${nested_table_mor2} ( + `k1` bigint(11) NULL, + `c_bool` array> NULL, + `c_tinyint` array> NULL, + `c_smallint` array> NULL, + `c_int` array> NULL, + `c_bigint` array> NULL, + `c_largeint` array> NULL, + `c_float` array> NULL, + `c_double` array> NULL, + `c_decimal` array> NULL, + `c_decimalv3` array> NULL, + `c_date` array> NULL, + `c_datetime` array> NULL, + `c_datev2` array> NULL, + `c_datetimev2` array> NULL, + `c_char` array> NULL, + `c_varchar` array> NULL, + `c_string` array> NULL + ) ENGINE=OLAP + UNIQUE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + // define mor key table with map types with one nested scala + def nested_table_map_mor = "tbl_map_types_mor" + sql "DROP TABLE IF EXISTS ${nested_table_map_mor}" + sql """ + CREATE TABLE IF NOT EXISTS ${nested_table_map_mor} ( + `k1` bigint(11) NULL, + `c_bool` map NULL, + `c_tinyint` map NULL, + `c_smallint` map NULL, + `c_int` map NULL, + `c_bigint` map NULL, + `c_largeint` map NULL, + `c_float` map NULL, + `c_double` map NULL, + `c_decimal` map NULL, + `c_decimalv3` map NULL, + `c_date` map NULL, + `c_datetime` map NULL, + `c_datev2` map NULL, + `c_datetimev2` map NULL, + `c_char` map NULL, + `c_varchar` map NULL, + `c_string` map NULL + ) ENGINE=OLAP + UNIQUE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + + // define mor key table with array nested map table types with one nested scala + def nested_table_array_map_mor = "tbl_array_map_types_mor" + sql "DROP TABLE IF EXISTS ${nested_table_array_map_mor}" + sql """ + CREATE TABLE IF NOT EXISTS ${nested_table_array_map_mor} ( + `k1` bigint(11) NULL, + `c_bool` array> NULL, + `c_tinyint` array> NULL, + `c_smallint` array> NULL, + `c_int` array> NULL, + `c_bigint` array> NULL, + `c_largeint` array> NULL, + `c_float` array> NULL, + `c_double` array> NULL, + `c_decimal` array> NULL, + `c_decimalv3` array> NULL, + `c_date` array> NULL, + `c_datetime` array> NULL, + `c_datev2` array> NULL, + `c_datetimev2` array> NULL, + `c_char` array> NULL, + `c_varchar` array> NULL, + `c_string` array> NULL + ) ENGINE=OLAP + UNIQUE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + // define mor key table with map nested value array table types with one nested scala + def nested_table_map_array_mor = "tbl_map_array_types_mor" + sql "DROP TABLE IF EXISTS ${nested_table_map_array_mor}" + sql """ + CREATE TABLE IF NOT EXISTS ${nested_table_map_array_mor} ( + `k1` bigint(11) NULL, + `c_bool` map> NULL, + `c_tinyint` map> NULL, + `c_smallint` map> NULL, + `c_int` map> NULL, + `c_bigint` map> NULL, + `c_largeint` map> NULL, + `c_float` map> NULL, + `c_double` map> NULL, + `c_decimal` map> NULL, + `c_decimalv3` map> NULL, + `c_date` map> NULL, + `c_datetime` map> NULL, + `c_datev2` map> NULL, + `c_datetimev2` map> NULL, + `c_char` map> NULL, + `c_varchar` map> NULL, + `c_string` map> NULL + ) ENGINE=OLAP + UNIQUE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + // define mow key table with nested table types with one nested scala + def nested_table_mow = "tbl_array_nested_types_mow" + sql "DROP TABLE IF EXISTS ${nested_table_mow}" + sql """ + CREATE TABLE IF NOT EXISTS ${nested_table_mow} ( + `k1` bigint(11) NULL, + `c_bool` array NULL, + `c_tinyint` array NULL, + `c_smallint` array NULL, + `c_int` array NULL, + `c_bigint` array NULL, + `c_largeint` array NULL, + `c_float` array NULL, + `c_double` array NULL, + `c_decimal` array NULL, + `c_decimalv3` array NULL, + `c_date` array NULL, + `c_datetime` array NULL, + `c_datev2` array NULL, + `c_datetimev2` array NULL, + `c_char` array NULL, + `c_varchar` array NULL, + `c_string` array NULL + ) ENGINE=OLAP + UNIQUE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1", "enable_unique_key_merge_on_write" = "true"); + """ + + // define mow key table with nested table types with two nested scala + def nested_table_mow2 = "tbl_array_nested_types_mow2" + sql "DROP TABLE IF EXISTS ${nested_table_mow2}" + sql """ + CREATE TABLE IF NOT EXISTS ${nested_table_mow2} ( + `k1` bigint(11) NULL, + `c_bool` array> NULL, + `c_tinyint` array> NULL, + `c_smallint` array> NULL, + `c_int` array> NULL, + `c_bigint` array> NULL, + `c_largeint` array> NULL, + `c_float` array> NULL, + `c_double` array> NULL, + `c_decimal` array> NULL, + `c_decimalv3` array> NULL, + `c_date` array> NULL, + `c_datetime` array> NULL, + `c_datev2` array> NULL, + `c_datetimev2` array> NULL, + `c_char` array> NULL, + `c_varchar` array> NULL, + `c_string` array> NULL + ) ENGINE=OLAP + UNIQUE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1", "enable_unique_key_merge_on_write" = "true"); + """ + + // define mow key table with map types with one nested scala + def nested_table_map_mow = "tbl_map_types_mow" + sql "DROP TABLE IF EXISTS ${nested_table_map_mow}" + sql """ + CREATE TABLE IF NOT EXISTS ${nested_table_map_mow} ( + `k1` bigint(11) NULL, + `c_bool` map NULL, + `c_tinyint` map NULL, + `c_smallint` map NULL, + `c_int` map NULL, + `c_bigint` map NULL, + `c_largeint` map NULL, + `c_float` map NULL, + `c_double` map NULL, + `c_decimal` map NULL, + `c_decimalv3` map NULL, + `c_date` map NULL, + `c_datetime` map NULL, + `c_datev2` map NULL, + `c_datetimev2` map NULL, + `c_char` map NULL, + `c_varchar` map NULL, + `c_string` map NULL + ) ENGINE=OLAP + UNIQUE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1", "enable_unique_key_merge_on_write" = "true"); + """ + + + // define mow key table with array nested map table types with one nested scala + def nested_table_array_map_mow = "tbl_array_map_types_mow" + sql "DROP TABLE IF EXISTS ${nested_table_array_map_mow}" + sql """ + CREATE TABLE IF NOT EXISTS ${nested_table_array_map_mow} ( + `k1` bigint(11) NULL, + `c_bool` array> NULL, + `c_tinyint` array> NULL, + `c_smallint` array> NULL, + `c_int` array> NULL, + `c_bigint` array> NULL, + `c_largeint` array> NULL, + `c_float` array> NULL, + `c_double` array> NULL, + `c_decimal` array> NULL, + `c_decimalv3` array> NULL, + `c_date` array> NULL, + `c_datetime` array> NULL, + `c_datev2` array> NULL, + `c_datetimev2` array> NULL, + `c_char` array> NULL, + `c_varchar` array> NULL, + `c_string` array> NULL + ) ENGINE=OLAP + UNIQUE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1", "enable_unique_key_merge_on_write" = "true"); + """ + + // define dup key table with map nested value array table types with one nested scala + def nested_table_map_array_mow = "tbl_map_array_types_mow" + sql "DROP TABLE IF EXISTS ${nested_table_map_array_mow}" + sql """ + CREATE TABLE IF NOT EXISTS ${nested_table_map_array_mow} ( + `k1` bigint(11) NULL, + `c_bool` map> NULL, + `c_tinyint` map> NULL, + `c_smallint` map> NULL, + `c_int` map> NULL, + `c_bigint` map> NULL, + `c_largeint` map> NULL, + `c_float` map> NULL, + `c_double` map> NULL, + `c_decimal` map> NULL, + `c_decimalv3` map> NULL, + `c_date` map> NULL, + `c_datetime` map> NULL, + `c_datev2` map> NULL, + `c_datetimev2` map> NULL, + `c_char` map> NULL, + `c_varchar` map> NULL, + `c_string` map> NULL + ) ENGINE=OLAP + UNIQUE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES("replication_num" = "1", "enable_unique_key_merge_on_write" = "true"); + """ + +} diff --git a/regression-test/suites/datatype_p0/nested_types/negative_cases/test_nested_types_insert_into_with_dup_table.groovy b/regression-test/suites/datatype_p0/nested_types/negative_cases/test_nested_types_insert_into_with_dup_table.groovy new file mode 100644 index 00000000000000..e8e9b363da1290 --- /dev/null +++ b/regression-test/suites/datatype_p0/nested_types/negative_cases/test_nested_types_insert_into_with_dup_table.groovy @@ -0,0 +1,442 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite("test_nested_types_insert_into_with_dup_table", "p0") { + sql 'use regression_test_datatype_p0_nested_types' + sql 'set enable_nereids_planner=false' + + // test action for scala to array with scala type + // current we support char family to insert nested type + test { + sql "insert into tbl_array_nested_types_dup (c_bool) select c_bool from tbl_scalar_types_dup" + exception "java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=ARRAY" + } + + test { + sql "insert into tbl_array_nested_types_dup (c_tinyint) select c_tinyint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_dup (c_smallint) select c_smallint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_dup (c_int) select c_int from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_dup (c_largeint) select c_largeint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_dup (c_float) select c_float from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_dup (c_double) select c_double from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_dup (c_decimal) select c_decimal from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_dup (c_decimalv3) select c_decimalv3 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_dup (c_date) select c_date from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_dup (c_datetime) select c_datetime from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_dup (c_datev2) select c_datev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_dup (c_datetimev2) select c_datetimev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_dup (c_char) select c_char from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_nested_types_dup (c_varchar) select c_varchar from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_nested_types_dup (c_string) select c_string from tbl_scalar_types_dup" + exception null + } + + qt_sql_nested_table_dup_c """select count() from tbl_array_nested_types_dup;""" + + // test action for scala to array with array-scala type + test { + sql "insert into tbl_array_nested_types_dup2 (c_bool) select c_bool from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_dup2 (c_tinyint) select c_tinyint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_dup2 (c_smallint) select c_smallint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_dup2 (c_int) select c_int from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_dup2 (c_largeint) select c_largeint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_dup2 (c_float) select c_float from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_dup2 (c_double) select c_double from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_dup2 (c_decimal) select c_decimal from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_dup2 (c_decimalv3) select c_decimalv3 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_dup2 (c_date) select c_date from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_dup2 (c_datetime) select c_datetime from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_dup2 (c_datev2) select c_datev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_dup2 (c_datetimev2) select c_datetimev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_dup2 (c_char) select c_char from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_nested_types_dup2 (c_varchar) select c_varchar from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_nested_types_dup2 (c_string) select c_string from tbl_scalar_types_dup" + exception null + } + + qt_sql_nested_table_dup2_c """select count() from tbl_array_nested_types_dup2;""" + + + // test action for scala to map with map-scala-scala type + test { + sql "insert into tbl_map_types_dup (c_bool) select c_bool from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=MAP") + } + + test { + sql "insert into tbl_map_types_dup (c_tinyint) select c_tinyint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=MAP") + } + + test { + sql "insert into tbl_map_types_dup (c_smallint) select c_smallint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=MAP") + } + + test { + sql "insert into tbl_map_types_dup (c_int) select c_int from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=MAP") + } + + test { + sql "insert into tbl_map_types_dup (c_largeint) select c_largeint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=MAP") + } + + test { + sql "insert into tbl_map_types_dup (c_float) select c_float from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=MAP") + } + + test { + sql "insert into tbl_map_types_dup (c_double) select c_double from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=MAP") + } + + test { + sql "insert into tbl_map_types_dup (c_decimal) select c_decimal from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=MAP") + } + + test { + sql "insert into tbl_map_types_dup (c_decimalv3) select c_decimalv3 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=MAP") + } + + test { + sql "insert into tbl_map_types_dup (c_date) select c_date from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=MAP") + } + + test { + sql "insert into tbl_map_types_dup (c_datetime) select c_datetime from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=MAP") + } + + test { + sql "insert into tbl_map_types_dup (c_datev2) select c_datev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=MAP") + } + + test { + sql "insert into tbl_map_types_dup (c_datetimev2) select c_datetimev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=MAP") + } + + test { + sql "insert into tbl_map_types_dup (c_char) select c_char from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_map_types_dup (c_varchar) select c_varchar from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_map_types_dup (c_string) select c_string from tbl_scalar_types_dup" + exception null + } + + qt_sql_nested_table_map_dup_c """select count() from tbl_map_types_dup;""" + + // test action for scala to array with map-scala-scala type + test { + sql "insert into tbl_array_map_types_dup (c_bool) select c_bool from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_dup (c_tinyint) select c_tinyint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_dup (c_smallint) select c_smallint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_dup (c_int) select c_int from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_dup (c_largeint) select c_largeint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_dup (c_float) select c_float from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_dup (c_double) select c_double from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_dup (c_decimal) select c_decimal from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_dup (c_decimalv3) select c_decimalv3 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_dup (c_date) select c_date from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_dup (c_datetime) select c_datetime from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_dup (c_datev2) select c_datev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_dup (c_datetimev2) select c_datetimev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_dup (c_char) select c_char from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_map_types_dup (c_varchar) select c_varchar from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_map_types_dup (c_string) select c_string from tbl_scalar_types_dup" + exception null + } + + qt_sql_nested_table_array_map_dup_c """select count() from tbl_array_map_types_dup;""" + + // test action for map with scala array-scala + // test action for scala to array with array-scala type + test { + sql "insert into tbl_map_array_types_dup (c_bool) select c_bool from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_dup (c_tinyint) select c_tinyint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_dup (c_smallint) select c_smallint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_dup (c_int) select c_int from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_dup (c_largeint) select c_largeint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_dup (c_float) select c_float from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_dup (c_double) select c_double from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_dup (c_decimal) select c_decimal from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_dup (c_decimalv3) select c_decimalv3 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_dup (c_date) select c_date from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_dup (c_datetime) select c_datetime from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_dup (c_datev2) select c_datev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_dup (c_datetimev2) select c_datetimev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_dup (c_char) select c_char from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_map_array_types_dup (c_varchar) select c_varchar from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_map_array_types_dup (c_string) select c_string from tbl_scalar_types_dup" + exception null + } + + qt_sql_nested_table_map_array_dup_c """select count() from tbl_map_array_types_dup;""" + +} diff --git a/regression-test/suites/datatype_p0/nested_types/negative_cases/test_nested_types_insert_into_with_unique_table.groovy b/regression-test/suites/datatype_p0/nested_types/negative_cases/test_nested_types_insert_into_with_unique_table.groovy new file mode 100644 index 00000000000000..2e7f8257c69477 --- /dev/null +++ b/regression-test/suites/datatype_p0/nested_types/negative_cases/test_nested_types_insert_into_with_unique_table.groovy @@ -0,0 +1,861 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite("test_nested_types_insert_into_with_unique_table", "p0") { + sql 'use regression_test_datatype_p0_nested_types' + sql 'set enable_nereids_planner=false' + + // test action for scala to array with scala type + // current we support char family to insert nested type + // mor table test + test { + sql "insert into tbl_array_nested_types_mor (c_bool) select c_bool from tbl_scalar_types_dup" + exception "java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=ARRAY" + } + + test { + sql "insert into tbl_array_nested_types_mor (c_tinyint) select c_tinyint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mor (c_smallint) select c_smallint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mor (c_int) select c_int from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mor (c_largeint) select c_largeint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mor (c_float) select c_float from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mor (c_double) select c_double from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mor (c_decimal) select c_decimal from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mor (c_decimalv3) select c_decimalv3 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mor (c_date) select c_date from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mor (c_datetime) select c_datetime from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mor (c_datev2) select c_datev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mor (c_datetimev2) select c_datetimev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mor (c_char) select c_char from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_nested_types_mor (c_varchar) select c_varchar from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_nested_types_mor (c_string) select c_string from tbl_scalar_types_dup" + exception null + } + + qt_sql_nested_table_mor_c """select count() from tbl_array_nested_types_mor;""" + + // test action for scala to array with array-scala type + test { + sql "insert into tbl_array_nested_types_mor2 (c_bool) select c_bool from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mor2 (c_tinyint) select c_tinyint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mor2 (c_smallint) select c_smallint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mor2 (c_int) select c_int from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mor2 (c_largeint) select c_largeint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mor2 (c_float) select c_float from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mor2 (c_double) select c_double from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mor2 (c_decimal) select c_decimal from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mor2 (c_decimalv3) select c_decimalv3 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mor2 (c_date) select c_date from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mor2 (c_datetime) select c_datetime from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mor2 (c_datev2) select c_datev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mor2 (c_datetimev2) select c_datetimev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mor2 (c_char) select c_char from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_nested_types_mor2 (c_varchar) select c_varchar from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_nested_types_mor2 (c_string) select c_string from tbl_scalar_types_dup" + exception null + } + + qt_sql_nested_table_mor2_c """select count() from tbl_array_nested_types_mor2;""" + + + // test action for scala to map with map-scala-scala type + test { + sql "insert into tbl_map_types_mor (c_bool) select c_bool from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mor (c_tinyint) select c_tinyint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mor (c_smallint) select c_smallint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mor (c_int) select c_int from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mor (c_largeint) select c_largeint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mor (c_float) select c_float from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mor (c_double) select c_double from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mor (c_decimal) select c_decimal from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mor (c_decimalv3) select c_decimalv3 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mor (c_date) select c_date from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mor (c_datetime) select c_datetime from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mor (c_datev2) select c_datev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mor (c_datetimev2) select c_datetimev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mor (c_char) select c_char from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_map_types_mor (c_varchar) select c_varchar from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_map_types_mor (c_string) select c_string from tbl_scalar_types_dup" + exception null + } + + qt_sql_nested_table_map_mor_c """select count() from tbl_map_types_mor;""" + + // test action for scala to array with map-scala-scala type + test { + sql "insert into tbl_array_map_types_mor (c_bool) select c_bool from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mor (c_tinyint) select c_tinyint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mor (c_smallint) select c_smallint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mor (c_int) select c_int from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mor (c_largeint) select c_largeint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mor (c_float) select c_float from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mor (c_double) select c_double from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mor (c_decimal) select c_decimal from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mor (c_decimalv3) select c_decimalv3 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mor (c_date) select c_date from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mor (c_datetime) select c_datetime from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mor (c_datev2) select c_datev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mor (c_datetimev2) select c_datetimev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mor (c_char) select c_char from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_map_types_mor (c_varchar) select c_varchar from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_map_types_mor (c_string) select c_string from tbl_scalar_types_dup" + exception null + } + + qt_sql_nested_table_array_map_mor_c """select count() from tbl_array_map_types_mor;""" + + // test action for map with scala array-scala + // test action for scala to array with array-scala type + test { + sql "insert into tbl_map_array_types_mor (c_bool) select c_bool from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mor (c_tinyint) select c_tinyint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mor (c_smallint) select c_smallint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mor (c_int) select c_int from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mor (c_largeint) select c_largeint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mor (c_float) select c_float from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mor (c_double) select c_double from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mor (c_decimal) select c_decimal from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mor (c_decimalv3) select c_decimalv3 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mor (c_date) select c_date from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mor (c_datetime) select c_datetime from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mor (c_datev2) select c_datev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mor (c_datetimev2) select c_datetimev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mor (c_char) select c_char from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_map_array_types_mor (c_varchar) select c_varchar from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_map_array_types_mor (c_string) select c_string from tbl_scalar_types_dup" + exception null + } + + qt_sql_nested_table_map_array_mor_c """select count() from tbl_map_array_types_mor;""" + + + // mow table test + test { + sql "insert into tbl_array_nested_types_mow (c_bool) select c_bool from tbl_scalar_types_dup" + exception "java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=ARRAY" + } + + test { + sql "insert into tbl_array_nested_types_mow (c_tinyint) select c_tinyint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mow (c_smallint) select c_smallint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mow (c_int) select c_int from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mow (c_largeint) select c_largeint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mow (c_float) select c_float from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mow (c_double) select c_double from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mow (c_decimal) select c_decimal from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mow (c_decimalv3) select c_decimalv3 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mow (c_date) select c_date from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mow (c_datetime) select c_datetime from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mow (c_datev2) select c_datev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mow (c_datetimev2) select c_datetimev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY") + } + + test { + sql "insert into tbl_array_nested_types_mow (c_char) select c_char from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_nested_types_mow (c_varchar) select c_varchar from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_nested_types_mow (c_string) select c_string from tbl_scalar_types_dup" + exception null + } + + qt_sql_nested_table_mow_c """select count() from tbl_array_nested_types_mow;""" + + // test action for scala to array with array-scala type + test { + sql "insert into tbl_array_nested_types_mow2 (c_bool) select c_bool from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mow2 (c_tinyint) select c_tinyint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mow2 (c_smallint) select c_smallint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mow2 (c_int) select c_int from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mow2 (c_largeint) select c_largeint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mow2 (c_float) select c_float from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mow2 (c_double) select c_double from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mow2 (c_decimal) select c_decimal from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mow2 (c_decimalv3) select c_decimalv3 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mow2 (c_date) select c_date from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mow2 (c_datetime) select c_datetime from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mow2 (c_datev2) select c_datev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mow2 (c_datetimev2) select c_datetimev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_nested_types_mow2 (c_char) select c_char from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_nested_types_mow2 (c_varchar) select c_varchar from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_nested_types_mow2 (c_string) select c_string from tbl_scalar_types_dup" + exception null + } + + qt_sql_nested_table_mow2_c """select count() from tbl_array_nested_types_mow2;""" + + + // test action for scala to map with map-scala-scala type + test { + sql "insert into tbl_map_types_mow (c_bool) select c_bool from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mow (c_tinyint) select c_tinyint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mow (c_smallint) select c_smallint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mow (c_int) select c_int from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mow (c_largeint) select c_largeint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mow (c_float) select c_float from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mow (c_double) select c_double from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mow (c_decimal) select c_decimal from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mow (c_decimalv3) select c_decimalv3 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mow (c_date) select c_date from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mow (c_datetime) select c_datetime from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mow (c_datev2) select c_datev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mow (c_datetimev2) select c_datetimev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=MAP") + } + + test { + sql "insert into tbl_map_types_mow (c_char) select c_char from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_map_types_mow (c_varchar) select c_varchar from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_map_types_mow (c_string) select c_string from tbl_scalar_types_dup" + exception null + } + + qt_sql_nested_table_map_mow_c """select count() from tbl_map_types_mow;""" + + // test action for scala to array with map-scala-scala type + test { + sql "insert into tbl_array_map_types_mow (c_bool) select c_bool from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mow (c_tinyint) select c_tinyint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mow (c_smallint) select c_smallint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mow (c_int) select c_int from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mow (c_largeint) select c_largeint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mow (c_float) select c_float from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mow (c_double) select c_double from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mow (c_decimal) select c_decimal from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mow (c_decimalv3) select c_decimalv3 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mow (c_date) select c_date from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mow (c_datetime) select c_datetime from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mow (c_datev2) select c_datev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mow (c_datetimev2) select c_datetimev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY>") + } + + test { + sql "insert into tbl_array_map_types_mow (c_char) select c_char from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_map_types_mow (c_varchar) select c_varchar from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_array_map_types_mow (c_string) select c_string from tbl_scalar_types_dup" + exception null + } + + qt_sql_nested_table_array_map_mow_c """select count() from tbl_array_map_types_mow;""" + + // test action for map with scala array-scala + // test action for scala to array with array-scala type + test { + sql "insert into tbl_map_array_types_mow (c_bool) select c_bool from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mow (c_tinyint) select c_tinyint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mow (c_smallint) select c_smallint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mow (c_int) select c_int from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mow (c_largeint) select c_largeint from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mow (c_float) select c_float from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mow (c_double) select c_double from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mow (c_decimal) select c_decimal from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mow (c_decimalv3) select c_decimalv3 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mow (c_date) select c_date from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mow (c_datetime) select c_datetime from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mow (c_datev2) select c_datev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mow (c_datetimev2) select c_datetimev2 from tbl_scalar_types_dup" + exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=MAP>") + } + + test { + sql "insert into tbl_map_array_types_mow (c_char) select c_char from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_map_array_types_mow (c_varchar) select c_varchar from tbl_scalar_types_dup" + exception null + } + + test { + sql "insert into tbl_map_array_types_mow (c_string) select c_string from tbl_scalar_types_dup" + exception null + } + + qt_sql_nested_table_map_array_mow_c """select count() from tbl_map_array_types_mow;""" + +} diff --git a/regression-test/suites/datatype_p0/nested_types/test_nested_types_insert_into_with_duplicat_table.groovy b/regression-test/suites/datatype_p0/nested_types/test_nested_types_insert_into_with_duplicat_table.groovy deleted file mode 100644 index e0bc822bdc42c5..00000000000000 --- a/regression-test/suites/datatype_p0/nested_types/test_nested_types_insert_into_with_duplicat_table.groovy +++ /dev/null @@ -1,647 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -import org.codehaus.groovy.runtime.IOGroovyMethods - -suite("test_nested_types_insert_into_with_duplicat_table", "p0") { - sql """ADMIN SET FRONTEND CONFIG ('disable_nested_complex_type' = 'false')""" - def dataFile = """test_scalar_types_100.csv""" - - // define dup key table1 with scala types - def scala_table_dup = "tbl_scalar_types_dup" - sql "DROP TABLE IF EXISTS ${scala_table_dup}" - sql """ - CREATE TABLE IF NOT EXISTS ${scala_table_dup} ( - `k1` bigint(11) NULL, - `c_bool` boolean NULL, - `c_tinyint` tinyint(4) NULL, - `c_smallint` smallint(6) NULL, - `c_int` int(11) NULL, - `c_bigint` bigint(20) NULL, - `c_largeint` largeint(40) NULL, - `c_float` float NULL, - `c_double` double NULL, - `c_decimal` decimal(20, 3) NULL, - `c_decimalv3` decimalv3(20, 3) NULL, - `c_date` date NULL, - `c_datetime` datetime NULL, - `c_datev2` datev2 NULL, - `c_datetimev2` datetimev2(0) NULL, - `c_char` char(15) NULL, - `c_varchar` varchar(100) NULL, - `c_string` text NULL - ) ENGINE=OLAP - DUPLICATE KEY(`k1`) - COMMENT 'OLAP' - DISTRIBUTED BY HASH(`k1`) BUCKETS 10 - PROPERTIES("replication_num" = "1"); - """ - - // load data - streamLoad { - table scala_table_dup - file dataFile - time 60000 - - check { result, exception, startTime, endTime -> - if (exception != null) { - throw exception - } - log.info("Stream load result: ${result}".toString()) - def json = parseJson(result) - assertEquals(100, json.NumberTotalRows) - assertEquals(100, json.NumberLoadedRows) - } - } - - // insert two NULL rows - sql """INSERT INTO ${scala_table_dup} VALUES (NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, - NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)""" - sql """INSERT INTO ${scala_table_dup} VALUES (NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, - NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)""" - - - // define dup key table with nested table types with one nested scala - def nested_table_dup = "tbl_array_nested_types_dup" - sql "DROP TABLE IF EXISTS ${nested_table_dup}" - sql """ - CREATE TABLE IF NOT EXISTS ${nested_table_dup} ( - `k1` bigint(11) NULL, - `c_bool` array NULL, - `c_tinyint` array NULL, - `c_smallint` array NULL, - `c_int` array NULL, - `c_bigint` array NULL, - `c_largeint` array NULL, - `c_float` array NULL, - `c_double` array NULL, - `c_decimal` array NULL, - `c_decimalv3` array NULL, - `c_date` array NULL, - `c_datetime` array NULL, - `c_datev2` array NULL, - `c_datetimev2` array NULL, - `c_char` array NULL, - `c_varchar` array NULL, - `c_string` array NULL - ) ENGINE=OLAP - DUPLICATE KEY(`k1`) - COMMENT 'OLAP' - DISTRIBUTED BY HASH(`k1`) BUCKETS 10 - PROPERTIES("replication_num" = "1"); - """ - - // define dup key table with nested table types with two nested scala - def nested_table_dup2 = "tbl_array_nested_types_dup2" - sql "DROP TABLE IF EXISTS ${nested_table_dup2}" - sql """ - CREATE TABLE IF NOT EXISTS ${nested_table_dup2} ( - `k1` bigint(11) NULL, - `c_bool` array> NULL, - `c_tinyint` array> NULL, - `c_smallint` array> NULL, - `c_int` array> NULL, - `c_bigint` array> NULL, - `c_largeint` array> NULL, - `c_float` array> NULL, - `c_double` array> NULL, - `c_decimal` array> NULL, - `c_decimalv3` array> NULL, - `c_date` array> NULL, - `c_datetime` array> NULL, - `c_datev2` array> NULL, - `c_datetimev2` array> NULL, - `c_char` array> NULL, - `c_varchar` array> NULL, - `c_string` array> NULL - ) ENGINE=OLAP - DUPLICATE KEY(`k1`) - COMMENT 'OLAP' - DISTRIBUTED BY HASH(`k1`) BUCKETS 10 - PROPERTIES("replication_num" = "1"); - """ - - // define dup key table with map types with one nested scala - def nested_table_map_dup = "tbl_map_types_dup" - sql "DROP TABLE IF EXISTS ${nested_table_map_dup}" - sql """ - CREATE TABLE IF NOT EXISTS ${nested_table_map_dup} ( - `k1` bigint(11) NULL, - `c_bool` map NULL, - `c_tinyint` map NULL, - `c_smallint` map NULL, - `c_int` map NULL, - `c_bigint` map NULL, - `c_largeint` map NULL, - `c_float` map NULL, - `c_double` map NULL, - `c_decimal` map NULL, - `c_decimalv3` map NULL, - `c_date` map NULL, - `c_datetime` map NULL, - `c_datev2` map NULL, - `c_datetimev2` map NULL, - `c_char` map NULL, - `c_varchar` map NULL, - `c_string` map NULL - ) ENGINE=OLAP - DUPLICATE KEY(`k1`) - COMMENT 'OLAP' - DISTRIBUTED BY HASH(`k1`) BUCKETS 10 - PROPERTIES("replication_num" = "1"); - """ - - - // define dup key table with array nested map table types with one nested scala - def nested_table_array_map_dup = "tbl_array_map_types_dup" - sql "DROP TABLE IF EXISTS ${nested_table_array_map_dup}" - sql """ - CREATE TABLE IF NOT EXISTS ${nested_table_array_map_dup} ( - `k1` bigint(11) NULL, - `c_bool` array> NULL, - `c_tinyint` array> NULL, - `c_smallint` array> NULL, - `c_int` array> NULL, - `c_bigint` array> NULL, - `c_largeint` array> NULL, - `c_float` array> NULL, - `c_double` array> NULL, - `c_decimal` array> NULL, - `c_decimalv3` array> NULL, - `c_date` array> NULL, - `c_datetime` array> NULL, - `c_datev2` array> NULL, - `c_datetimev2` array> NULL, - `c_char` array> NULL, - `c_varchar` array> NULL, - `c_string` array> NULL - ) ENGINE=OLAP - DUPLICATE KEY(`k1`) - COMMENT 'OLAP' - DISTRIBUTED BY HASH(`k1`) BUCKETS 10 - PROPERTIES("replication_num" = "1"); - """ - - // define dup key table with map nested value array table types with one nested scala - def nested_table_map_array_dup = "tbl_map_array_types_dup" - sql "DROP TABLE IF EXISTS ${nested_table_map_array_dup}" - sql """ - CREATE TABLE IF NOT EXISTS ${nested_table_map_array_dup} ( - `k1` bigint(11) NULL, - `c_bool` map> NULL, - `c_tinyint` map> NULL, - `c_smallint` map> NULL, - `c_int` map> NULL, - `c_bigint` map> NULL, - `c_largeint` map> NULL, - `c_float` map> NULL, - `c_double` map> NULL, - `c_decimal` map> NULL, - `c_decimalv3` map> NULL, - `c_date` map> NULL, - `c_datetime` map> NULL, - `c_datev2` map> NULL, - `c_datetimev2` map> NULL, - `c_char` map> NULL, - `c_varchar` map> NULL, - `c_string` map> NULL - ) ENGINE=OLAP - DUPLICATE KEY(`k1`) - COMMENT 'OLAP' - DISTRIBUTED BY HASH(`k1`) BUCKETS 10 - PROPERTIES("replication_num" = "1"); - """ - - // test action for scala to array with scala type - // current we support char family to insert nested type - test { - sql "insert into ${nested_table_dup} (c_bool) select c_bool from ${scala_table_dup}" - exception "java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=ARRAY" - } - - test { - sql "insert into ${nested_table_dup} (c_tinyint) select c_tinyint from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=ARRAY") - } - - test { - sql "insert into ${nested_table_dup} (c_smallint) select c_smallint from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=ARRAY") - } - - test { - sql "insert into ${nested_table_dup} (c_int) select c_int from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=ARRAY") - } - - test { - sql "insert into ${nested_table_dup} (c_largeint) select c_largeint from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=ARRAY") - } - - test { - sql "insert into ${nested_table_dup} (c_float) select c_float from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=ARRAY") - } - - test { - sql "insert into ${nested_table_dup} (c_double) select c_double from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=ARRAY") - } - - test { - sql "insert into ${nested_table_dup} (c_decimal) select c_decimal from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY") - } - - test { - sql "insert into ${nested_table_dup} (c_decimalv3) select c_decimalv3 from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY") - } - - test { - sql "insert into ${nested_table_dup} (c_date) select c_date from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY") - } - - test { - sql "insert into ${nested_table_dup} (c_datetime) select c_datetime from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY") - } - - test { - sql "insert into ${nested_table_dup} (c_datev2) select c_datev2 from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY") - } - - test { - sql "insert into ${nested_table_dup} (c_datetimev2) select c_datetimev2 from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY") - } - - test { - sql "insert into ${nested_table_dup} (c_char) select c_char from ${scala_table_dup}" - exception null - } - - test { - sql "insert into ${nested_table_dup} (c_varchar) select c_varchar from ${scala_table_dup}" - exception null - } - - test { - sql "insert into ${nested_table_dup} (c_string) select c_string from ${scala_table_dup}" - exception null - } - - qt_sql_nested_table_dup_c """select count() from ${nested_table_dup};""" - - // test action for scala to array with array-scala type - test { - sql "insert into ${nested_table_dup2} (c_bool) select c_bool from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_dup2} (c_tinyint) select c_tinyint from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_dup2} (c_smallint) select c_smallint from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_dup2} (c_int) select c_int from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_dup2} (c_largeint) select c_largeint from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_dup2} (c_float) select c_float from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_dup2} (c_double) select c_double from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_dup2} (c_decimal) select c_decimal from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_dup2} (c_decimalv3) select c_decimalv3 from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_dup2} (c_date) select c_date from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_dup2} (c_datetime) select c_datetime from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_dup2} (c_datev2) select c_datev2 from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_dup2} (c_datetimev2) select c_datetimev2 from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_dup2} (c_char) select c_char from ${scala_table_dup}" - exception null - } - - test { - sql "insert into ${nested_table_dup2} (c_varchar) select c_varchar from ${scala_table_dup}" - exception null - } - - test { - sql "insert into ${nested_table_dup2} (c_string) select c_string from ${scala_table_dup}" - exception null - } - - qt_sql_nested_table_dup2_c """select count() from ${nested_table_dup2};""" - - - // test action for scala to map with map-scala-scala type - test { - sql "insert into ${nested_table_map_dup} (c_bool) select c_bool from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=MAP") - } - - test { - sql "insert into ${nested_table_map_dup} (c_tinyint) select c_tinyint from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=MAP") - } - - test { - sql "insert into ${nested_table_map_dup} (c_smallint) select c_smallint from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=MAP") - } - - test { - sql "insert into ${nested_table_map_dup} (c_int) select c_int from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=MAP") - } - - test { - sql "insert into ${nested_table_map_dup} (c_largeint) select c_largeint from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=MAP") - } - - test { - sql "insert into ${nested_table_map_dup} (c_float) select c_float from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=MAP") - } - - test { - sql "insert into ${nested_table_map_dup} (c_double) select c_double from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=MAP") - } - - test { - sql "insert into ${nested_table_map_dup} (c_decimal) select c_decimal from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=MAP") - } - - test { - sql "insert into ${nested_table_map_dup} (c_decimalv3) select c_decimalv3 from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=MAP") - } - - test { - sql "insert into ${nested_table_map_dup} (c_date) select c_date from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=MAP") - } - - test { - sql "insert into ${nested_table_map_dup} (c_datetime) select c_datetime from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=MAP") - } - - test { - sql "insert into ${nested_table_map_dup} (c_datev2) select c_datev2 from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=MAP") - } - - test { - sql "insert into ${nested_table_map_dup} (c_datetimev2) select c_datetimev2 from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=MAP") - } - - test { - sql "insert into ${nested_table_map_dup} (c_char) select c_char from ${scala_table_dup}" - exception null - } - - test { - sql "insert into ${nested_table_map_dup} (c_varchar) select c_varchar from ${scala_table_dup}" - exception null - } - - test { - sql "insert into ${nested_table_map_dup} (c_string) select c_string from ${scala_table_dup}" - exception null - } - - qt_sql_nested_table_map_dup_c """select count() from ${nested_table_map_dup};""" - - // test action for scala to array with map-scala-scala type - test { - sql "insert into ${nested_table_array_map_dup} (c_bool) select c_bool from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_array_map_dup} (c_tinyint) select c_tinyint from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_array_map_dup} (c_smallint) select c_smallint from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_array_map_dup} (c_int) select c_int from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_array_map_dup} (c_largeint) select c_largeint from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_array_map_dup} (c_float) select c_float from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_array_map_dup} (c_double) select c_double from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_array_map_dup} (c_decimal) select c_decimal from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_array_map_dup} (c_decimalv3) select c_decimalv3 from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_array_map_dup} (c_date) select c_date from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_array_map_dup} (c_datetime) select c_datetime from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_array_map_dup} (c_datev2) select c_datev2 from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_array_map_dup} (c_datetimev2) select c_datetimev2 from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=ARRAY>") - } - - test { - sql "insert into ${nested_table_array_map_dup} (c_char) select c_char from ${scala_table_dup}" - exception null - } - - test { - sql "insert into ${nested_table_array_map_dup} (c_varchar) select c_varchar from ${scala_table_dup}" - exception null - } - - test { - sql "insert into ${nested_table_array_map_dup} (c_string) select c_string from ${scala_table_dup}" - exception null - } - - qt_sql_nested_table_array_map_dup_c """select count() from ${nested_table_array_map_dup};""" - - // test action for map with scala array-scala - // test action for scala to array with array-scala type - test { - sql "insert into ${nested_table_map_array_dup} (c_bool) select c_bool from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type BOOLEAN to target type=MAP>") - } - - test { - sql "insert into ${nested_table_map_array_dup} (c_tinyint) select c_tinyint from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type TINYINT to target type=MAP>") - } - - test { - sql "insert into ${nested_table_map_array_dup} (c_smallint) select c_smallint from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type SMALLINT to target type=MAP>") - } - - test { - sql "insert into ${nested_table_map_array_dup} (c_int) select c_int from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type INT to target type=MAP>") - } - - test { - sql "insert into ${nested_table_map_array_dup} (c_largeint) select c_largeint from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type LARGEINT to target type=MAP>") - } - - test { - sql "insert into ${nested_table_map_array_dup} (c_float) select c_float from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type FLOAT to target type=MAP>") - } - - test { - sql "insert into ${nested_table_map_array_dup} (c_double) select c_double from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DOUBLE to target type=MAP>") - } - - test { - sql "insert into ${nested_table_map_array_dup} (c_decimal) select c_decimal from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=MAP>") - } - - test { - sql "insert into ${nested_table_map_array_dup} (c_decimalv3) select c_decimalv3 from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DECIMALV3(20, 3) to target type=MAP>") - } - - test { - sql "insert into ${nested_table_map_array_dup} (c_date) select c_date from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=MAP>") - } - - test { - sql "insert into ${nested_table_map_array_dup} (c_datetime) select c_datetime from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=MAP>") - } - - test { - sql "insert into ${nested_table_map_array_dup} (c_datev2) select c_datev2 from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATEV2 to target type=MAP>") - } - - test { - sql "insert into ${nested_table_map_array_dup} (c_datetimev2) select c_datetimev2 from ${scala_table_dup}" - exception("java.sql.SQLException: errCode = 2, detailMessage = can not cast from origin type DATETIMEV2(0) to target type=MAP>") - } - - test { - sql "insert into ${nested_table_map_array_dup} (c_char) select c_char from ${scala_table_dup}" - exception null - } - - test { - sql "insert into ${nested_table_map_array_dup} (c_varchar) select c_varchar from ${scala_table_dup}" - exception null - } - - test { - sql "insert into ${nested_table_map_array_dup} (c_string) select c_string from ${scala_table_dup}" - exception null - } - - qt_sql_nested_table_map_array_dup_c """select count() from ${nested_table_map_array_dup};""" - -} diff --git a/regression-test/suites/demo_p0/httpTest_action.groovy b/regression-test/suites/demo_p0/httpTest_action.groovy index 6d03e081f8f131..3120a92b5f864c 100644 --- a/regression-test/suites/demo_p0/httpTest_action.groovy +++ b/regression-test/suites/demo_p0/httpTest_action.groovy @@ -24,7 +24,7 @@ suite("http_test_action") { def backendIdToBackendIP = [:] def backendIdToBackendBrpcPort = [:] for (String[] backend in backends) { - if (backend[8].equals("true")) { + if (backend[9].equals("true")) { backendIdToBackendIP.put(backend[0], backend[1]) backendIdToBackendBrpcPort.put(backend[0], backend[5]) } diff --git a/regression-test/suites/external_table_p0/tvf/test_backends_tvf.groovy b/regression-test/suites/external_table_p0/tvf/test_backends_tvf.groovy index be497ee25a881f..2f6f774ad85332 100644 --- a/regression-test/suites/external_table_p0/tvf/test_backends_tvf.groovy +++ b/regression-test/suites/external_table_p0/tvf/test_backends_tvf.groovy @@ -19,7 +19,7 @@ suite("test_backends_tvf","p0,external,tvf,external_docker") { List> table = sql """ select * from backends(); """ assertTrue(table.size() > 0) - assertEquals(24, table[0].size) + assertEquals(25, table[0].size) // filter columns table = sql """ select BackendId, Host, Alive, TotalCapacity, Version, NodeRole from backends();""" diff --git a/regression-test/suites/external_table_p0/tvf/test_frontends_tvf.groovy b/regression-test/suites/external_table_p0/tvf/test_frontends_tvf.groovy index e247f8bdf1a1c0..0f7a4f1b2d3c21 100644 --- a/regression-test/suites/external_table_p0/tvf/test_frontends_tvf.groovy +++ b/regression-test/suites/external_table_p0/tvf/test_frontends_tvf.groovy @@ -19,7 +19,7 @@ suite("test_frontends_tvf","p0,external,tvf,external_docker") { List> table = sql """ select * from `frontends`(); """ assertTrue(table.size() > 0) - assertTrue(table[0].size == 17) + assertTrue(table[0].size == 18) // filter columns table = sql """ select Name from `frontends`();""" @@ -43,7 +43,7 @@ suite("test_frontends_tvf","p0,external,tvf,external_docker") { assertTrue(res[0][0] > 0) sql """ select Name, Host, EditLogPort - HttpPort, QueryPort, RpcPort, `Role`, IsMaster, ClusterId + HttpPort, QueryPort, RpcPort, ArrowFlightSqlPort, `Role`, IsMaster, ClusterId `Join`, Alive, ReplayedJournalId, LastHeartbeat IsHelper, ErrMsg, Version, CurrentConnected from frontends(); """ diff --git a/regression-test/suites/json_p0/test_json_load_and_function.groovy b/regression-test/suites/json_p0/test_json_load_and_function.groovy index 11b72ae486fd4e..6f728330722f38 100644 --- a/regression-test/suites/json_p0/test_json_load_and_function.groovy +++ b/regression-test/suites/json_p0/test_json_load_and_function.groovy @@ -18,6 +18,7 @@ import org.codehaus.groovy.runtime.IOGroovyMethods suite("test_json_load_and_function", "p0") { + // define a sql table def testTable = "tbl_test_json" def dataFile = "test_json.csv" diff --git a/regression-test/suites/jsonb_p0/test_jsonb_load_and_function.groovy b/regression-test/suites/jsonb_p0/test_jsonb_load_and_function.groovy index 821b349d40e7dd..46b64813e02c67 100644 --- a/regression-test/suites/jsonb_p0/test_jsonb_load_and_function.groovy +++ b/regression-test/suites/jsonb_p0/test_jsonb_load_and_function.groovy @@ -533,12 +533,6 @@ suite("test_jsonb_load_and_function", "p0") { qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.k2', null) FROM ${testTable} ORDER BY id""" qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.a1[0].k1', '\$.a1[0].k2', '\$.a1[2]') FROM ${testTable} ORDER BY id""" - qt_select """SELECT id, j, j->'\$.k1' FROM ${testTable} ORDER BY id""" - qt_select """SELECT id, j, j->'\$.[1]' FROM ${testTable} ORDER BY id""" - qt_select """SELECT id, j, j->null FROM ${testTable} ORDER BY id""" - qt_select """SELECT id, j, j->'\$.a1[0].k2' FROM ${testTable} ORDER BY id""" - qt_select """SELECT id, j, j->'\$.a1[0]'->'\$.k1' FROM ${testTable} ORDER BY id""" - //json_length qt_sql_json_length """SELECT json_length('1')""" qt_sql_json_length """SELECT json_length('true')""" diff --git a/regression-test/suites/load_p0/broker_load/test_array_load.groovy b/regression-test/suites/load_p0/broker_load/test_array_load.groovy index 5d7e04d883d0be..5ef0bf7d18d26d 100644 --- a/regression-test/suites/load_p0/broker_load/test_array_load.groovy +++ b/regression-test/suites/load_p0/broker_load/test_array_load.groovy @@ -249,12 +249,12 @@ suite("test_array_load", "load_p0") { brokerName =getBrokerName() hdfsUser = getHdfsUser() hdfsPasswd = getHdfsPasswd() - def hdfs_json_file_path = uploadToHdfs "broker_load/simple_object_array.json" - def hdfs_csv_file_path = uploadToHdfs "broker_load/simple_array.csv" - def hdfs_orc_file_path = uploadToHdfs "broker_load/simple_array.orc" + def hdfs_json_file_path = uploadToHdfs "load_p0/broker_load/simple_object_array.json" + def hdfs_csv_file_path = uploadToHdfs "load_p0/broker_load/simple_array.csv" + def hdfs_orc_file_path = uploadToHdfs "load_p0/broker_load/simple_array.orc" // orc file with native array(list) type - def hdfs_orc_file_path2 = uploadToHdfs "broker_load/simple_array_list_type.orc" - def hdfs_parquet_file_path = uploadToHdfs "broker_load/simple_array.parquet" + def hdfs_orc_file_path2 = uploadToHdfs "load_p0/broker_load/simple_array_list_type.orc" + def hdfs_parquet_file_path = uploadToHdfs "load_p0/broker_load/simple_array.parquet" // case5: import array data by hdfs and enable vectorized engine try { diff --git a/regression-test/suites/load_p0/broker_load/test_broker_load_with_where.groovy b/regression-test/suites/load_p0/broker_load/test_broker_load_with_where.groovy new file mode 100644 index 00000000000000..dd05905e4bbdf2 --- /dev/null +++ b/regression-test/suites/load_p0/broker_load/test_broker_load_with_where.groovy @@ -0,0 +1,162 @@ +// 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. + +suite("test_broker_load_with_where", "load_p0") { + // define a sql table + def testTable = "tbl_test_broker_load_with_where" + + def create_test_table = {testTablex -> + def result1 = sql """ + CREATE TABLE IF NOT EXISTS ${testTable} ( + `k1` BIGINT NOT NULL, + `k2` DATE NULL, + `k3` INT(11) NOT NULL, + `k4` INT(11) NOT NULL, + `v5` BIGINT SUM NULL DEFAULT "0" + ) ENGINE=OLAP + AGGREGATE KEY(`k1`, `k2`, `k3`, `k4`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 16 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "storage_format" = "V2" + ); + """ + + // DDL/DML return 1 row and 3 column, the only value is update row count + assertTrue(result1.size() == 1) + assertTrue(result1[0].size() == 1) + assertTrue(result1[0][0] == 0, "Create table should update 0 rows") + + // insert 1 row to check whether the table is ok + def result2 = sql """ INSERT INTO ${testTable} VALUES + (1,2023-09-01,1,1,1) + """ + assertTrue(result2.size() == 1) + assertTrue(result2[0].size() == 1) + assertTrue(result2[0][0] == 1, "Insert should update 1 rows") + } + + def load_from_hdfs_norm = {testTablex, label, hdfsFilePath, format, brokerName, hdfsUser, hdfsPasswd -> + def result1= sql """ + LOAD LABEL ${label} ( + DATA INFILE("${hdfsFilePath}") + INTO TABLE ${testTablex} + COLUMNS TERMINATED BY "," + FORMAT as "${format}" + ) + with BROKER "${brokerName}" ( + "username"="${hdfsUser}", + "password"="${hdfsPasswd}") + PROPERTIES ( + "timeout"="1200", + "max_filter_ratio"="0.1"); + """ + + assertTrue(result1.size() == 1) + assertTrue(result1[0].size() == 1) + assertTrue(result1[0][0] == 0, "Query OK, 0 rows affected") + } + + def load_from_hdfs_with_or_predicate = {testTablex, label, hdfsFilePath, format, brokerName, hdfsUser, hdfsPasswd -> + def result1= sql """ + LOAD LABEL ${label} ( + DATA INFILE("${hdfsFilePath}") + INTO TABLE ${testTablex} + COLUMNS TERMINATED BY "," + FORMAT as "${format}" + WHERE + k1 in (11001,11002) + and ( + k3 in (1) + or k4 in (1, 2) + ) + ) + with BROKER "${brokerName}" ( + "username"="${hdfsUser}", + "password"="${hdfsPasswd}") + PROPERTIES ( + "timeout"="1200", + "max_filter_ratio"="0.1"); + """ + assertTrue(result1.size() == 1) + assertTrue(result1[0].size() == 1) + assertTrue(result1[0][0] == 0, "Query OK, 0 rows affected") + } + + def check_load_result = {checklabel, testTablex -> + max_try_milli_secs = 10000 + while(max_try_milli_secs) { + result = sql "show load where label = '${checklabel}'" + if(result[0][2] == "FINISHED") { + //sql "sync" + qt_select "select * from ${testTablex} order by k1" + break + } else { + sleep(1000) // wait 1 second every time + max_try_milli_secs -= 1000 + if(max_try_milli_secs <= 0) { + assertEquals(1, 2) + } + } + } + } + + def check_data_correct = {table_name -> + sql "sync" + // select the table and check whether the data is correct + qt_select "select k1,k3,k4,sum(v5) from ${table_name} group by k1,k3,k4 order by k1,k3,k4" + } + + // if 'enableHdfs' in regression-conf.groovy has been set to true, + // the test will run these case as below. + if (enableHdfs()) { + brokerName = getBrokerName() + hdfsUser = getHdfsUser() + hdfsPasswd = getHdfsPasswd() + def hdfs_csv_file_path = uploadToHdfs "load_p0/broker_load/broker_load_with_where.csv" + //def hdfs_csv_file_path = "hdfs://ip:port/testfile" + + // case1: import csv data from hdfs with out where + try { + sql "DROP TABLE IF EXISTS ${testTable}" + create_test_table.call(testTable) + + def test_load_label = UUID.randomUUID().toString().replaceAll("-", "") + load_from_hdfs_norm.call(testTable, test_load_label, hdfs_csv_file_path, "csv", + brokerName, hdfsUser, hdfsPasswd) + + check_load_result.call(test_load_label, testTable) + } finally { + try_sql("DROP TABLE IF EXISTS ${testTable}") + } + + // case2: import csv data from hdfs with or predicate in where + try { + sql "DROP TABLE IF EXISTS ${testTable}" + create_test_table.call(testTable) + + def test_load_label = UUID.randomUUID().toString().replaceAll("-", "") + load_from_hdfs_with_or_predicate.call(testTable, test_load_label, hdfs_csv_file_path, "csv", + brokerName, hdfsUser, hdfsPasswd) + + check_load_result.call(test_load_label, testTable) + } finally { + try_sql("DROP TABLE IF EXISTS ${testTable}") + } + } +} diff --git a/regression-test/suites/load_p0/spark_load/test_spark_load.groovy b/regression-test/suites/load_p0/spark_load/test_spark_load.groovy index 6cc3df7381d86d..c798ad1fcf78f5 100644 --- a/regression-test/suites/load_p0/spark_load/test_spark_load.groovy +++ b/regression-test/suites/load_p0/spark_load/test_spark_load.groovy @@ -130,8 +130,8 @@ suite("test_spark_load", "p0") { // if 'enableHdfs' in regression-conf.groovy has been set to true, if (enableHdfs()) { - def hdfs_txt_file_path1 = uploadToHdfs "spark_load/all_types1.txt" - def hdfs_txt_file_path2 = uploadToHdfs "spark_load/all_types2.txt" + def hdfs_txt_file_path1 = uploadToHdfs "load_p0/spark_load/all_types1.txt" + def hdfs_txt_file_path2 = uploadToHdfs "load_p0/spark_load/all_types2.txt" try { sql "DROP TABLE IF EXISTS ${testTable}" sql "DROP TABLE IF EXISTS ${testTable2}" diff --git a/regression-test/suites/load_p0/stream_load/test_csv_split_line.groovy b/regression-test/suites/load_p0/stream_load/test_csv_split_line.groovy index c3b786bfc792f2..47bd8c3bbc9af9 100644 --- a/regression-test/suites/load_p0/stream_load/test_csv_split_line.groovy +++ b/regression-test/suites/load_p0/stream_load/test_csv_split_line.groovy @@ -108,6 +108,7 @@ suite("test_csv_split_line", "p0") { streamLoad { table "${tableName}2" set 'column_separator', 'hello' + set 'trim_double_quotes', 'true' file 'test_csv_split_line2.csv' } streamLoad { @@ -124,4 +125,44 @@ suite("test_csv_split_line", "p0") { sql """ drop table ${tableName}2; """ + sql """ DROP TABLE IF EXISTS ${tableName}3 """ + sql """ create table ${tableName}3 ( + `user_id` bigint(20) NULL, + `tag_type` varchar(20) NULL , + `tag_owner_id` bigint(20) NULL, + `tag_value` text NULL , + `deleted` tinyint(4) NULL , + `create_time` datetime NULL DEFAULT CURRENT_TIMESTAMP + ) ENGINE=OLAP + UNIQUE KEY(`user_id`, `tag_type`, `tag_owner_id`) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 20 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "is_being_synced" = "false", + "colocate_with" = "__global__crm_user_group", + "storage_format" = "V2", + "enable_unique_key_merge_on_write" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false" + ); + """ + + streamLoad { + table "${tableName}3" + set 'column_separator', '||' + file 'test_csv_split_line4.csv' + } + order_qt_sql """ + select * from ${tableName}3 order by user_id; + """ + + order_qt_sql """ + select * from ${tableName}3 where tag_value="" order by user_id; + """ + order_qt_sql """ + select * from ${tableName}3 where tag_value="" order by user_id; + """ + + sql """ drop table ${tableName}3; """ + } diff --git a/regression-test/suites/load_p0/stream_load/test_json_load.groovy b/regression-test/suites/load_p0/stream_load/test_json_load.groovy index 539eafe6ede797..7ad38874fbcb25 100644 --- a/regression-test/suites/load_p0/stream_load/test_json_load.groovy +++ b/regression-test/suites/load_p0/stream_load/test_json_load.groovy @@ -622,7 +622,7 @@ suite("test_json_load", "p0") { brokerName =getBrokerName() hdfsUser = getHdfsUser() hdfsPasswd = getHdfsPasswd() - def hdfs_file_path = uploadToHdfs "stream_load/simple_object_json.json" + def hdfs_file_path = uploadToHdfs "load_p0/stream_load/simple_object_json.json" def format = "json" // case18: import json use pre-filter exprs diff --git a/regression-test/suites/load_p0/stream_load/test_map_load_and_compaction.groovy b/regression-test/suites/load_p0/stream_load/test_map_load_and_compaction.groovy index c5eb2689bdedaa..4a3b99c1d06f2d 100644 --- a/regression-test/suites/load_p0/stream_load/test_map_load_and_compaction.groovy +++ b/regression-test/suites/load_p0/stream_load/test_map_load_and_compaction.groovy @@ -134,7 +134,7 @@ suite("test_map_load_and_compaction", "p0") { backends = sql """ show backends; """ assertTrue(backends.size() > 0) for (String[] b : backends) { - assertEquals("true", b[8]) + assertEquals("true", b[9]) } } finally { try_sql("DROP TABLE IF EXISTS ${testTable}") diff --git a/regression-test/suites/nereids_function_p0/load.groovy b/regression-test/suites/nereids_function_p0/load.groovy index 2f7b61b6874fe7..c403302d0b1811 100644 --- a/regression-test/suites/nereids_function_p0/load.groovy +++ b/regression-test/suites/nereids_function_p0/load.groovy @@ -223,4 +223,4 @@ suite("load") { sql """ insert into fn_test_not_nullable select * from fn_test where id is not null """ -} \ No newline at end of file +} diff --git a/regression-test/suites/nereids_function_p0/scalar_function/J.groovy b/regression-test/suites/nereids_function_p0/scalar_function/J.groovy new file mode 100644 index 00000000000000..39e63485df5baa --- /dev/null +++ b/regression-test/suites/nereids_function_p0/scalar_function/J.groovy @@ -0,0 +1,535 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite("nereids_scalar_fn_J") { + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + // define a sql table + def testTable = "tbl_test_jsonb" + def dataFile = "test_jsonb.csv" + + sql "DROP TABLE IF EXISTS ${testTable}" + + sql """ + CREATE TABLE IF NOT EXISTS ${testTable} ( + id INT, + j JSONB + ) + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + // load the jsonb data from csv file + streamLoad { + table testTable + + file dataFile // import csv file + time 10000 // limit inflight 10s + set 'strict_mode', 'true' + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + def (code, out, err) = curl("GET", json.ErrorURL) + log.info("error result: " + out) + + assertEquals("fail", json.Status.toLowerCase()) + assertTrue(json.Message.contains("too many filtered rows")) + assertEquals(25, json.NumberTotalRows) + assertEquals(18, json.NumberLoadedRows) + assertEquals(7, json.NumberFilteredRows) + assertTrue(json.LoadBytes > 0) + log.info("url: " + json.ErrorURL) + } + } + + // load the jsonb data from csv file + // success with header 'max_filter_ratio: 0.3' + streamLoad { + table testTable + + // set http request header params + set 'max_filter_ratio', '0.3' + file dataFile // import csv file + time 10000 // limit inflight 10s + set 'strict_mode', 'true' + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + def (code, out, err) = curl("GET", json.ErrorURL) + log.info("error result: " + out) + + assertEquals("success", json.Status.toLowerCase()) + assertEquals(25, json.NumberTotalRows) + assertEquals(18, json.NumberLoadedRows) + assertEquals(7, json.NumberFilteredRows) + assertTrue(json.LoadBytes > 0) + } + } + + // check result + qt_select "SELECT * FROM ${testTable} ORDER BY id" + + // insert into valid json rows + sql """INSERT INTO ${testTable} VALUES(26, NULL)""" + sql """INSERT INTO ${testTable} VALUES(27, '{"k1":"v1", "k2": 200}')""" + sql """INSERT INTO ${testTable} VALUES(28, '{"a.b.c":{"k1.a1":"v31", "k2": 300},"a":"niu"}')""" + // int64 value + sql """INSERT INTO ${testTable} VALUES(29, '12524337771678448270')""" + // int64 min value + sql """INSERT INTO ${testTable} VALUES(30, '-9223372036854775808')""" + // int64 max value + sql """INSERT INTO ${testTable} VALUES(31, '18446744073709551615')""" + + // insert into invalid json rows with enable_insert_strict=true + // expect excepiton and no rows not changed + sql """ set enable_insert_strict = true """ + def success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, '')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(false, success) + success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, 'abc')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(false, success) + + // insert into invalid json rows with enable_insert_strict=false + // expect no excepiton but no rows not changed + sql """ set enable_insert_strict = false """ + success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, '')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(true, success) + success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, 'abc')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(true, success) + + qt_select "SELECT * FROM ${testTable} ORDER BY id" + + // jsonb_extract + qt_jsonb_extract_select "SELECT id, j, jsonb_extract(j, '\$') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.*') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.\"a.b.c\"') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.\"a.b.c\".\"k1.a1\"') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_multipath + qt_jsonb_extract_multipath "SELECT id, j, jsonb_extract(j, '\$', '\$.*', '\$.k1', '\$[0]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_string + qt_jsonb_extract_string_select "SELECT id, j, jsonb_extract_string(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_int + qt_jsonb_extract_int_select "SELECT id, j, jsonb_extract_int(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_bigint + qt_jsonb_extract_bigint_select "SELECT id, j, jsonb_extract_bigint(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + + // jsonb_extract_largeint + qt_jsonb_extract_largeint_select "SELECT id, j, jsonb_extract_largeint(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + + // jsonb_extract_double + qt_jsonb_extract_double_select "SELECT id, j, jsonb_extract_double(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_bool + qt_jsonb_extract_bool_select "SELECT id, j, jsonb_extract_bool(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_isnull + qt_jsonb_extract_isnull_select "SELECT id, j, jsonb_extract_isnull(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_exists_path + qt_jsonb_exists_path_select "SELECT id, j, jsonb_exists_path(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_type + qt_jsonb_type_select "SELECT id, j, jsonb_type(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_type(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_type(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_type(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + + // CAST from JSONB + qt_cast_from_select "SELECT id, j, CAST(j AS BOOLEAN) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS SMALLINT) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS INT) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS BIGINT) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS DOUBLE) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS STRING) FROM ${testTable} ORDER BY id" + + // CAST to JSONB + qt_cast_to_select "SELECT id, j, CAST(CAST(j AS BOOLEAN) AS JSONB) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS SMALLINT) AS JSONB) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS INT) AS JSONB) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS BIGINT) AS JSONB) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS DOUBLE) AS JSONB) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS STRING) AS JSONB) FROM ${testTable} ORDER BY id" + + qt_select """SELECT CAST(NULL AS JSONB)""" + qt_select """SELECT CAST('null' AS JSONB)""" + qt_select """SELECT CAST('true' AS JSONB)""" + qt_select """SELECT CAST('false' AS JSONB)""" + qt_select """SELECT CAST('100' AS JSONB)""" + qt_select """SELECT CAST('10000' AS JSONB)""" + qt_select """SELECT CAST('1000000000' AS JSONB)""" + qt_select """SELECT CAST('1152921504606846976' AS JSONB)""" + qt_select """SELECT CAST('6.18' AS JSONB)""" + qt_select """SELECT CAST('"abcd"' AS JSONB)""" + qt_select """SELECT CAST('{}' AS JSONB)""" + qt_select """SELECT CAST('{"k1":"v31", "k2": 300}' AS JSONB)""" + qt_select """SELECT CAST('[]' AS JSONB)""" + qt_select """SELECT CAST('[123, 456]' AS JSONB)""" + qt_select """SELECT CAST('["abc", "def"]' AS JSONB)""" + qt_select """SELECT CAST('[null, true, false, 100, 6.18, "abc"]' AS JSONB)""" + qt_select """SELECT CAST('[{"k1":"v41", "k2": 400}, 1, "a", 3.14]' AS JSONB)""" + qt_select """SELECT CAST('{"k1":"v31", "k2": 300, "a1": [{"k1":"v41", "k2": 400}, 1, "a", 3.14]}' AS JSONB)""" + qt_select """SELECT CAST("''" AS JSONB)""" + qt_select """SELECT CAST("'abc'" AS JSONB)""" + qt_select """SELECT CAST('abc' AS JSONB)""" + qt_select """SELECT CAST('100x' AS JSONB)""" + qt_select """SELECT CAST('6.a8' AS JSONB)""" + qt_select """SELECT CAST('{x' AS JSONB)""" + qt_select """SELECT CAST('[123, abc]' AS JSONB)""" + + qt_select """SELECT id, JSON_VALID(j) FROM ${testTable} ORDER BY id""" + qt_select """SELECT JSON_VALID('{"k1":"v31","k2":300}')""" + qt_select """SELECT JSON_VALID('invalid json')""" + qt_select """SELECT JSON_VALID(NULL)""" + + qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.k1') FROM ${testTable} ORDER BY id""" + qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.k2', '\$.[1]') FROM ${testTable} ORDER BY id""" + qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.k2', '\$.x.y') FROM ${testTable} ORDER BY id""" + qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.k2', null) FROM ${testTable} ORDER BY id""" + qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.a1[0].k1', '\$.a1[0].k2', '\$.a1[2]') FROM ${testTable} ORDER BY id""" +} diff --git a/regression-test/suites/nereids_p0/insert_into_table/partial_update.groovy b/regression-test/suites/nereids_p0/insert_into_table/partial_update.groovy index 472acfc0c5e02f..697bc21880929f 100644 --- a/regression-test/suites/nereids_p0/insert_into_table/partial_update.groovy +++ b/regression-test/suites/nereids_p0/insert_into_table/partial_update.groovy @@ -22,7 +22,6 @@ suite("nereids_partial_update_native_insert_stmt", "p0") { sql "set enable_fallback_to_original_planner=false;" sql "sync;" - // sql 'set enable_fallback_to_original_planner=false' def tableName = "nereids_partial_update_native_insert_stmt" sql """ DROP TABLE IF EXISTS ${tableName} """ sql """ @@ -100,6 +99,7 @@ suite("nereids_partial_update_native_insert_stmt", "p0") { sql """insert into ${tableName3} values(2, "doris2", 2000, 223, 1),(1, "doris", 1000, 123, 1);""" qt_3 """ select * from ${tableName3} order by id; """ sql "set enable_unique_key_partial_update=true;" + sql "set enable_insert_strict = false;" sql "sync;" // in partial update, the unmentioned columns should have default values or be nullable // but field `name` is not nullable and doesn't have default value diff --git a/regression-test/suites/nereids_p0/insert_into_table/unsupport_type.groovy b/regression-test/suites/nereids_p0/insert_into_table/unsupport_type.groovy index e62cfa777838f0..24e31bb064b7c1 100644 --- a/regression-test/suites/nereids_p0/insert_into_table/unsupport_type.groovy +++ b/regression-test/suites/nereids_p0/insert_into_table/unsupport_type.groovy @@ -23,14 +23,9 @@ suite("nereids_insert_unsupport_type") { sql 'set enable_nereids_dml=true' sql 'set enable_strict_consistency_dml=true' - test { - sql 'insert into json_t select id, kjson from src' - exception 'unsupported for Nereids' - } - sql 'set enable_fallback_to_original_planner=true' sql 'insert into map_t select id, kmintint from src' sql 'sync' sql 'select * from map_t' -} \ No newline at end of file +} diff --git a/regression-test/suites/nereids_p0/json_p0/test_json_load_and_function.groovy b/regression-test/suites/nereids_p0/json_p0/test_json_load_and_function.groovy new file mode 100644 index 00000000000000..14047a678b8dd1 --- /dev/null +++ b/regression-test/suites/nereids_p0/json_p0/test_json_load_and_function.groovy @@ -0,0 +1,466 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite("test_json_load_and_function", "p0") { + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + // define a sql table + def testTable = "tbl_test_json" + def dataFile = "test_json.csv" + + sql "DROP TABLE IF EXISTS ${testTable}" + + sql """ + CREATE TABLE IF NOT EXISTS ${testTable} ( + id INT, + j JSON + ) + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + // load the json data from csv file + streamLoad { + table testTable + + file dataFile // import csv file + time 10000 // limit inflight 10s + set 'strict_mode', 'true' + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + + def (code, out, err) = curl("GET", json.ErrorURL) + log.info("error result: " + out) + + assertEquals("fail", json.Status.toLowerCase()) + assertTrue(json.Message.contains("too many filtered rows")) + assertEquals(25, json.NumberTotalRows) + assertEquals(18, json.NumberLoadedRows) + assertEquals(7, json.NumberFilteredRows) + assertTrue(json.LoadBytes > 0) + log.info("url: " + json.ErrorURL) + } + } + + // load the json data from csv file + // success with header 'max_filter_ratio: 0.3' + streamLoad { + table testTable + + // set http request header params + set 'max_filter_ratio', '0.3' + file dataFile // import csv file + time 10000 // limit inflight 10s + set 'strict_mode', 'true' + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + + def (code, out, err) = curl("GET", json.ErrorURL) + log.info("error result: " + out) + + assertEquals("success", json.Status.toLowerCase()) + assertEquals(25, json.NumberTotalRows) + assertEquals(18, json.NumberLoadedRows) + assertEquals(7, json.NumberFilteredRows) + assertTrue(json.LoadBytes > 0) + } + } + + // check result + qt_select "SELECT * FROM ${testTable} ORDER BY id" + + // insert into valid json rows + sql """INSERT INTO ${testTable} VALUES(26, NULL)""" + sql """INSERT INTO ${testTable} VALUES(27, '{"k1":"v1", "k2": 200}')""" + sql """INSERT INTO ${testTable} VALUES(28, '{"a.b.c":{"k1.a1":"v31", "k2": 300},"a":"niu"}')""" + // int64 value + sql """INSERT INTO ${testTable} VALUES(29, '12524337771678448270')""" + // int64 min value + sql """INSERT INTO ${testTable} VALUES(30, '-9223372036854775808')""" + // int64 max value + sql """INSERT INTO ${testTable} VALUES(31, '18446744073709551615')""" + + // insert into invalid json rows with enable_insert_strict=true + // expect excepiton and no rows not changed + sql """ set enable_insert_strict = true """ + def success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, '')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(false, success) + success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, 'abc')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(false, success) + + // insert into invalid json rows with enable_insert_strict=false + // expect no excepiton but no rows not changed + sql """ set enable_insert_strict = false """ + success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, '')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(true, success) + success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, 'abc')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(true, success) + + qt_select "SELECT * FROM ${testTable} ORDER BY id" + + // json_extract + qt_select "SELECT id, j, jsonb_extract(j, '\$') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.*') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.\"a.b.c\"') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.\"a.b.c\".\"k1.a1\"') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + + // json_extract_string + qt_select "SELECT id, j, json_extract_string(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_string(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_string(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_string(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_string(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + + // json_extract_int + qt_select "SELECT id, j, json_extract_int(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_int(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_int(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_int(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_int(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + + // json_extract_bigint + qt_select "SELECT id, j, json_extract_bigint(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_bigint(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_bigint(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_bigint(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_bigint(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + + + // json_extract_largeint + qt_json_extract_largeint_select "SELECT id, j, json_extract_largeint(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_largeint(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_largeint(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + + // json_extract_double + qt_select "SELECT id, j, json_extract_double(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_double(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_double(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_double(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_double(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + + // json_extract_bool + qt_select "SELECT id, j, json_extract_bool(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_bool(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_bool(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_bool(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_bool(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + + // json_extract_isnull + qt_select "SELECT id, j, json_extract_isnull(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_isnull(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_isnull(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_isnull(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_isnull(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + + // json_exists_path + qt_select "SELECT id, j, json_exists_path(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_exists_path(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_exists_path(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_exists_path(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_exists_path(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + + // json_type + qt_select "SELECT id, j, json_type(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_type(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_type(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_type(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_type(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + + + // CAST from JSON + qt_select "SELECT id, j, CAST(j AS BOOLEAN) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS SMALLINT) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS INT) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS BIGINT) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS DOUBLE) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS STRING) FROM ${testTable} ORDER BY id" + + // CAST to JSON + qt_select "SELECT id, j, CAST(CAST(j AS BOOLEAN) AS JSON) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS SMALLINT) AS JSON) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS INT) AS JSON) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS BIGINT) AS JSON) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS DOUBLE) AS JSON) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS STRING) AS JSON) FROM ${testTable} ORDER BY id" + + qt_select """SELECT CAST(NULL AS JSON)""" + qt_select """SELECT CAST('null' AS JSON)""" + qt_select """SELECT CAST('true' AS JSON)""" + qt_select """SELECT CAST('false' AS JSON)""" + qt_select """SELECT CAST('100' AS JSON)""" + qt_select """SELECT CAST('10000' AS JSON)""" + qt_select """SELECT CAST('1000000000' AS JSON)""" + qt_select """SELECT CAST('1152921504606846976' AS JSON)""" + qt_select """SELECT CAST('6.18' AS JSON)""" + qt_select """SELECT CAST('"abcd"' AS JSON)""" + qt_select """SELECT CAST('{}' AS JSON)""" + qt_select """SELECT CAST('{"k1":"v31", "k2": 300}' AS JSON)""" + qt_select """SELECT CAST('[]' AS JSON)""" + qt_select """SELECT CAST('[123, 456]' AS JSON)""" + qt_select """SELECT CAST('["abc", "def"]' AS JSON)""" + qt_select """SELECT CAST('[null, true, false, 100, 6.18, "abc"]' AS JSON)""" + qt_select """SELECT CAST('[{"k1":"v41", "k2": 400}, 1, "a", 3.14]' AS JSON)""" + qt_select """SELECT CAST('{"k1":"v31", "k2": 300, "a1": [{"k1":"v41", "k2": 400}, 1, "a", 3.14]}' AS JSON)""" + qt_select """SELECT CAST("''" AS JSON)""" + qt_select """SELECT CAST("'abc'" AS JSON)""" + qt_select """SELECT CAST('abc' AS JSON)""" + qt_select """SELECT CAST('100x' AS JSON)""" + qt_select """SELECT CAST('6.a8' AS JSON)""" + qt_select """SELECT CAST('{x' AS JSON)""" + qt_select """SELECT CAST('[123, abc]' AS JSON)""" + + qt_select """SELECT id, JSON_VALID(j) FROM ${testTable} ORDER BY id""" + qt_select """SELECT JSON_VALID('{"k1":"v31","k2":300}')""" + qt_select """SELECT JSON_VALID('invalid json')""" + qt_select """SELECT JSON_VALID(NULL)""" + + qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.k1') FROM ${testTable} ORDER BY id""" + qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.k2', '\$.[1]') FROM ${testTable} ORDER BY id""" + qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.k2', '\$.x.y') FROM ${testTable} ORDER BY id""" + qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.k2', null) FROM ${testTable} ORDER BY id""" + qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.a1[0].k1', '\$.a1[0].k2', '\$.a1[2]') FROM ${testTable} ORDER BY id""" +} diff --git a/regression-test/suites/nereids_p0/json_p0/test_json_load_unique_key_and_function.groovy b/regression-test/suites/nereids_p0/json_p0/test_json_load_unique_key_and_function.groovy new file mode 100644 index 00000000000000..87916fddc29e0a --- /dev/null +++ b/regression-test/suites/nereids_p0/json_p0/test_json_load_unique_key_and_function.groovy @@ -0,0 +1,408 @@ +// 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. + +suite("test_json_unique_load_and_function", "p0") { + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + // define a sql table + def testTable = "tbl_test_json_unique" + def dataFile = "test_json_unique_key.csv" + + sql "DROP TABLE IF EXISTS ${testTable}" + + sql """ + CREATE TABLE IF NOT EXISTS ${testTable} ( + id INT, + j JSON + ) + UNIQUE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + // load the json data from csv file + // fail by default for invalid data rows + streamLoad { + table testTable + + file dataFile // import csv file + time 10000 // limit inflight 10s + set 'strict_mode', 'true' + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + assertEquals("fail", json.Status.toLowerCase()) + assertTrue(json.Message.contains("too many filtered rows")) + assertEquals(75, json.NumberTotalRows) + assertEquals(54, json.NumberLoadedRows) + assertEquals(21, json.NumberFilteredRows) + assertTrue(json.LoadBytes > 0) + } + } + + // load the json data from csv file + // success with header 'max_filter_ratio: 0.3' + streamLoad { + table testTable + + // set http request header params + set 'max_filter_ratio', '0.3' + file dataFile // import csv file + time 10000 // limit inflight 10s + set 'strict_mode', 'true' + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + assertEquals(75, json.NumberTotalRows) + assertEquals(54, json.NumberLoadedRows) + assertEquals(21, json.NumberFilteredRows) + assertTrue(json.LoadBytes > 0) + } + } + + sql "sync" + // check result + qt_select "SELECT * FROM ${testTable} ORDER BY id" + + // insert into valid json rows + sql """INSERT INTO ${testTable} VALUES(26, NULL)""" + sql """INSERT INTO ${testTable} VALUES(27, '{"k1":"v1", "k2": 200}')""" + sql """INSERT INTO ${testTable} VALUES(28, '{"a.b.c":{"k1.a1":"v31", "k2": 300},"a":"niu"}')""" + + // insert into invalid json rows with enable_insert_strict=true + // expect excepiton and no rows not changed + sql """ set enable_insert_strict = true """ + def success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, '')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(false, success) + success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, 'abc')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(false, success) + + // insert into invalid json rows with enable_insert_strict=false + // expect no excepiton but no rows not changed + sql """ set enable_insert_strict = false """ + success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, '')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(true, success) + success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, 'abc')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(true, success) + + qt_select "SELECT * FROM ${testTable} ORDER BY id" + + // jsonb_extract + qt_select "SELECT id, j, jsonb_extract(j, '\$') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.*') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.\"a.b.c\"') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.\"a.b.c\".\"k1.a1\"') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + + // json_extract_string + qt_select "SELECT id, j, json_extract_string(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_string(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_string(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_string(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_string(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_string(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + + // json_extract_int + qt_select "SELECT id, j, json_extract_int(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_int(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_int(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_int(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_int(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_int(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + + // json_extract_bigint + qt_select "SELECT id, j, json_extract_bigint(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_bigint(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_bigint(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_bigint(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_bigint(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bigint(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + + + // json_extract_double + qt_select "SELECT id, j, json_extract_double(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_double(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_double(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_double(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_double(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_double(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + + // json_extract_bool + qt_select "SELECT id, j, json_extract_bool(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_bool(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_bool(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_bool(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_bool(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_bool(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + + // json_extract_isnull + qt_select "SELECT id, j, json_extract_isnull(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_isnull(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_isnull(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_isnull(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_extract_isnull(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_extract_isnull(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + + // json_exists_path + qt_select "SELECT id, j, json_exists_path(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_exists_path(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_exists_path(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_exists_path(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_exists_path(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_exists_path(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + + // json_type + qt_select "SELECT id, j, json_type(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_type(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_type(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_type(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, json_type(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, json_type(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + + + // CAST from JSON + qt_select "SELECT id, j, CAST(j AS BOOLEAN) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS SMALLINT) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS INT) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS BIGINT) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS DOUBLE) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS STRING) FROM ${testTable} ORDER BY id" + + // CAST to JSON + qt_select "SELECT id, j, CAST(CAST(j AS BOOLEAN) AS JSON) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS SMALLINT) AS JSON) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS INT) AS JSON) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS BIGINT) AS JSON) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS DOUBLE) AS JSON) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS STRING) AS JSON) FROM ${testTable} ORDER BY id" + + qt_select """SELECT CAST(NULL AS JSON)""" + qt_select """SELECT CAST('null' AS JSON)""" + qt_select """SELECT CAST('true' AS JSON)""" + qt_select """SELECT CAST('false' AS JSON)""" + qt_select """SELECT CAST('100' AS JSON)""" + qt_select """SELECT CAST('10000' AS JSON)""" + qt_select """SELECT CAST('1000000000' AS JSON)""" + qt_select """SELECT CAST('1152921504606846976' AS JSON)""" + qt_select """SELECT CAST('6.18' AS JSON)""" + qt_select """SELECT CAST('"abcd"' AS JSON)""" + qt_select """SELECT CAST('{}' AS JSON)""" + qt_select """SELECT CAST('{"k1":"v31", "k2": 300}' AS JSON)""" + qt_select """SELECT CAST('[]' AS JSON)""" + qt_select """SELECT CAST('[123, 456]' AS JSON)""" + qt_select """SELECT CAST('["abc", "def"]' AS JSON)""" + qt_select """SELECT CAST('[null, true, false, 100, 6.18, "abc"]' AS JSON)""" + qt_select """SELECT CAST('[{"k1":"v41", "k2": 400}, 1, "a", 3.14]' AS JSON)""" + qt_select """SELECT CAST('{"k1":"v31", "k2": 300, "a1": [{"k1":"v41", "k2": 400}, 1, "a", 3.14]}' AS JSON)""" + qt_select """SELECT CAST("''" AS JSON)""" + qt_select """SELECT CAST("'abc'" AS JSON)""" + qt_select """SELECT CAST('abc' AS JSON)""" + qt_select """SELECT CAST('100x' AS JSON)""" + qt_select """SELECT CAST('6.a8' AS JSON)""" + qt_select """SELECT CAST('{x' AS JSON)""" + qt_select """SELECT CAST('[123, abc]' AS JSON)""" + +} diff --git a/regression-test/suites/nereids_p0/jsonb_p0/test_jsonb_load_and_function.groovy b/regression-test/suites/nereids_p0/jsonb_p0/test_jsonb_load_and_function.groovy new file mode 100644 index 00000000000000..33258d6bec5bbc --- /dev/null +++ b/regression-test/suites/nereids_p0/jsonb_p0/test_jsonb_load_and_function.groovy @@ -0,0 +1,535 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.codehaus.groovy.runtime.IOGroovyMethods + +suite("test_jsonb_load_and_function", "p0") { + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + // define a sql table + def testTable = "tbl_test_jsonb" + def dataFile = "test_jsonb.csv" + + sql "DROP TABLE IF EXISTS ${testTable}" + + sql """ + CREATE TABLE IF NOT EXISTS ${testTable} ( + id INT, + j JSONB + ) + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + // load the jsonb data from csv file + streamLoad { + table testTable + + file dataFile // import csv file + time 10000 // limit inflight 10s + set 'strict_mode', 'true' + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + def (code, out, err) = curl("GET", json.ErrorURL) + log.info("error result: " + out) + + assertEquals("fail", json.Status.toLowerCase()) + assertTrue(json.Message.contains("too many filtered rows")) + assertEquals(25, json.NumberTotalRows) + assertEquals(18, json.NumberLoadedRows) + assertEquals(7, json.NumberFilteredRows) + assertTrue(json.LoadBytes > 0) + log.info("url: " + json.ErrorURL) + } + } + + // load the jsonb data from csv file + // success with header 'max_filter_ratio: 0.3' + streamLoad { + table testTable + + // set http request header params + set 'max_filter_ratio', '0.3' + file dataFile // import csv file + time 10000 // limit inflight 10s + set 'strict_mode', 'true' + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + def (code, out, err) = curl("GET", json.ErrorURL) + log.info("error result: " + out) + + assertEquals("success", json.Status.toLowerCase()) + assertEquals(25, json.NumberTotalRows) + assertEquals(18, json.NumberLoadedRows) + assertEquals(7, json.NumberFilteredRows) + assertTrue(json.LoadBytes > 0) + } + } + + // check result + qt_select "SELECT * FROM ${testTable} ORDER BY id" + + // insert into valid json rows + sql """INSERT INTO ${testTable} VALUES(26, NULL)""" + sql """INSERT INTO ${testTable} VALUES(27, '{"k1":"v1", "k2": 200}')""" + sql """INSERT INTO ${testTable} VALUES(28, '{"a.b.c":{"k1.a1":"v31", "k2": 300},"a":"niu"}')""" + // int64 value + sql """INSERT INTO ${testTable} VALUES(29, '12524337771678448270')""" + // int64 min value + sql """INSERT INTO ${testTable} VALUES(30, '-9223372036854775808')""" + // int64 max value + sql """INSERT INTO ${testTable} VALUES(31, '18446744073709551615')""" + + // insert into invalid json rows with enable_insert_strict=true + // expect excepiton and no rows not changed + sql """ set enable_insert_strict = true """ + def success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, '')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(false, success) + success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, 'abc')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(false, success) + + // insert into invalid json rows with enable_insert_strict=false + // expect no excepiton but no rows not changed + sql """ set enable_insert_strict = false """ + success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, '')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(true, success) + success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, 'abc')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(true, success) + + qt_select "SELECT * FROM ${testTable} ORDER BY id" + + // jsonb_extract + qt_jsonb_extract_select "SELECT id, j, jsonb_extract(j, '\$') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.*') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.\"a.b.c\"') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.\"a.b.c\".\"k1.a1\"') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_multipath + qt_jsonb_extract_multipath "SELECT id, j, jsonb_extract(j, '\$', '\$.*', '\$.k1', '\$[0]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_string + qt_jsonb_extract_string_select "SELECT id, j, jsonb_extract_string(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_int + qt_jsonb_extract_int_select "SELECT id, j, jsonb_extract_int(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_bigint + qt_jsonb_extract_bigint_select "SELECT id, j, jsonb_extract_bigint(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + + // jsonb_extract_largeint + qt_jsonb_extract_largeint_select "SELECT id, j, jsonb_extract_largeint(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + + // jsonb_extract_double + qt_jsonb_extract_double_select "SELECT id, j, jsonb_extract_double(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_bool + qt_jsonb_extract_bool_select "SELECT id, j, jsonb_extract_bool(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_isnull + qt_jsonb_extract_isnull_select "SELECT id, j, jsonb_extract_isnull(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_exists_path + qt_jsonb_exists_path_select "SELECT id, j, jsonb_exists_path(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_type + qt_jsonb_type_select "SELECT id, j, jsonb_type(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_type(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_type(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_type(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + + // CAST from JSONB + qt_cast_from_select "SELECT id, j, CAST(j AS BOOLEAN) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS SMALLINT) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS INT) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS BIGINT) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS DOUBLE) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS STRING) FROM ${testTable} ORDER BY id" + + // CAST to JSONB + qt_cast_to_select "SELECT id, j, CAST(CAST(j AS BOOLEAN) AS JSONB) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS SMALLINT) AS JSONB) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS INT) AS JSONB) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS BIGINT) AS JSONB) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS DOUBLE) AS JSONB) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS STRING) AS JSONB) FROM ${testTable} ORDER BY id" + + qt_select """SELECT CAST(NULL AS JSONB)""" + qt_select """SELECT CAST('null' AS JSONB)""" + qt_select """SELECT CAST('true' AS JSONB)""" + qt_select """SELECT CAST('false' AS JSONB)""" + qt_select """SELECT CAST('100' AS JSONB)""" + qt_select """SELECT CAST('10000' AS JSONB)""" + qt_select """SELECT CAST('1000000000' AS JSONB)""" + qt_select """SELECT CAST('1152921504606846976' AS JSONB)""" + qt_select """SELECT CAST('6.18' AS JSONB)""" + qt_select """SELECT CAST('"abcd"' AS JSONB)""" + qt_select """SELECT CAST('{}' AS JSONB)""" + qt_select """SELECT CAST('{"k1":"v31", "k2": 300}' AS JSONB)""" + qt_select """SELECT CAST('[]' AS JSONB)""" + qt_select """SELECT CAST('[123, 456]' AS JSONB)""" + qt_select """SELECT CAST('["abc", "def"]' AS JSONB)""" + qt_select """SELECT CAST('[null, true, false, 100, 6.18, "abc"]' AS JSONB)""" + qt_select """SELECT CAST('[{"k1":"v41", "k2": 400}, 1, "a", 3.14]' AS JSONB)""" + qt_select """SELECT CAST('{"k1":"v31", "k2": 300, "a1": [{"k1":"v41", "k2": 400}, 1, "a", 3.14]}' AS JSONB)""" + qt_select """SELECT CAST("''" AS JSONB)""" + qt_select """SELECT CAST("'abc'" AS JSONB)""" + qt_select """SELECT CAST('abc' AS JSONB)""" + qt_select """SELECT CAST('100x' AS JSONB)""" + qt_select """SELECT CAST('6.a8' AS JSONB)""" + qt_select """SELECT CAST('{x' AS JSONB)""" + qt_select """SELECT CAST('[123, abc]' AS JSONB)""" + + qt_select """SELECT id, JSON_VALID(j) FROM ${testTable} ORDER BY id""" + qt_select """SELECT JSON_VALID('{"k1":"v31","k2":300}')""" + qt_select """SELECT JSON_VALID('invalid json')""" + qt_select """SELECT JSON_VALID(NULL)""" + + qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.k1') FROM ${testTable} ORDER BY id""" + qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.k2', '\$.[1]') FROM ${testTable} ORDER BY id""" + qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.k2', '\$.x.y') FROM ${testTable} ORDER BY id""" + qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.k2', null) FROM ${testTable} ORDER BY id""" + qt_select """SELECT id, j, JSON_EXTRACT(j, '\$.a1[0].k1', '\$.a1[0].k2', '\$.a1[2]') FROM ${testTable} ORDER BY id""" +} diff --git a/regression-test/suites/nereids_p0/jsonb_p0/test_jsonb_load_unique_key_and_function.groovy b/regression-test/suites/nereids_p0/jsonb_p0/test_jsonb_load_unique_key_and_function.groovy new file mode 100644 index 00000000000000..013f5d6fab0581 --- /dev/null +++ b/regression-test/suites/nereids_p0/jsonb_p0/test_jsonb_load_unique_key_and_function.groovy @@ -0,0 +1,480 @@ +// 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. + +suite("test_jsonb_unique_load_and_function", "p0") { + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + // define a sql table + def testTable = "tbl_test_jsonb_unique" + def dataFile = "test_jsonb_unique_key.csv" + + sql "DROP TABLE IF EXISTS ${testTable}" + + sql """ + CREATE TABLE IF NOT EXISTS ${testTable} ( + id INT, + j JSONB + ) + UNIQUE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + // load the jsonb data from csv file + // fail by default for invalid data rows + streamLoad { + table testTable + + file dataFile // import csv file + time 10000 // limit inflight 10s + set 'strict_mode', 'true' + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + assertEquals("fail", json.Status.toLowerCase()) + assertTrue(json.Message.contains("too many filtered rows")) + assertEquals(75, json.NumberTotalRows) + assertEquals(54, json.NumberLoadedRows) + assertEquals(21, json.NumberFilteredRows) + assertTrue(json.LoadBytes > 0) + } + } + + // load the jsonb data from csv file + // success with header 'max_filter_ratio: 0.3' + streamLoad { + table testTable + + // set http request header params + set 'max_filter_ratio', '0.3' + file dataFile // import csv file + time 10000 // limit inflight 10s + set 'strict_mode', 'true' + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + assertEquals(75, json.NumberTotalRows) + assertEquals(54, json.NumberLoadedRows) + assertEquals(21, json.NumberFilteredRows) + assertTrue(json.LoadBytes > 0) + } + } + + sql "sync" + // check result + qt_select "SELECT * FROM ${testTable} ORDER BY id" + + // insert into valid json rows + sql """INSERT INTO ${testTable} VALUES(26, NULL)""" + sql """INSERT INTO ${testTable} VALUES(27, '{"k1":"v1", "k2": 200}')""" + sql """INSERT INTO ${testTable} VALUES(28, '{"a.b.c":{"k1.a1":"v31", "k2": 300},"a":"niu"}')""" + + // insert into invalid json rows with enable_insert_strict=true + // expect excepiton and no rows not changed + sql """ set enable_insert_strict = true """ + def success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, '')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(false, success) + success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, 'abc')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(false, success) + + // insert into invalid json rows with enable_insert_strict=false + // expect no excepiton but no rows not changed + sql """ set enable_insert_strict = false """ + success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, '')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(true, success) + success = true + try { + sql """INSERT INTO ${testTable} VALUES(26, 'abc')""" + } catch(Exception ex) { + logger.info("""INSERT INTO ${testTable} invalid json failed: """ + ex) + success = false + } + assertEquals(true, success) + + qt_select "SELECT * FROM ${testTable} ORDER BY id" + + // jsonb_extract + qt_jsonb_extract_select "SELECT id, j, jsonb_extract(j, '\$') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.*') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.\"a.b.c\"') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.\"a.b.c\".\"k1.a1\"') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + + // jsonb_extract_multipath + qt_jsonb_extract_multipath "SELECT id, j, jsonb_extract(j, '\$', '\$.*', '\$.k1', '\$[0]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_string + qt_jsonb_extract_string_select "SELECT id, j, jsonb_extract_string(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_string(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_int + qt_jsonb_extract_int_select "SELECT id, j, jsonb_extract_int(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_int(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_bigint + qt_jsonb_extract_bigint_select "SELECT id, j, jsonb_extract_bigint(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + + // jsonb_extract_double + qt_jsonb_extract_double_select "SELECT id, j, jsonb_extract_double(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_double(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_bool + qt_jsonb_extract_bool_select "SELECT id, j, jsonb_extract_bool(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_bool(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_extract_isnull + qt_jsonb_extract_isnull_select "SELECT id, j, jsonb_extract_isnull(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_extract_isnull(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_exists_path + qt_jsonb_exists_path_select "SELECT id, j, jsonb_exists_path(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_exists_path(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + // jsonb_type + qt_jsonb_type_select "SELECT id, j, jsonb_type(j, '\$') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_type(j, '\$.k1') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.k2') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_type(j, '\$[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[5]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[6]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$[10]') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_type(j, '\$.a1') FROM ${testTable} ORDER BY id" + + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[3]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[4]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[last]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, jsonb_type(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id" + + + // CAST from JSONB + qt_cast_from_select "SELECT id, j, CAST(j AS BOOLEAN) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS SMALLINT) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS INT) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS BIGINT) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS DOUBLE) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(j AS STRING) FROM ${testTable} ORDER BY id" + + // CAST to JSONB + qt_cast_to_select "SELECT id, j, CAST(CAST(j AS BOOLEAN) AS JSONB) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS SMALLINT) AS JSONB) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS INT) AS JSONB) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS BIGINT) AS JSONB) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS DOUBLE) AS JSONB) FROM ${testTable} ORDER BY id" + qt_select "SELECT id, j, CAST(CAST(j AS STRING) AS JSONB) FROM ${testTable} ORDER BY id" + + qt_select """SELECT CAST(NULL AS JSONB)""" + qt_select """SELECT CAST('null' AS JSONB)""" + qt_select """SELECT CAST('true' AS JSONB)""" + qt_select """SELECT CAST('false' AS JSONB)""" + qt_select """SELECT CAST('100' AS JSONB)""" + qt_select """SELECT CAST('10000' AS JSONB)""" + qt_select """SELECT CAST('1000000000' AS JSONB)""" + qt_select """SELECT CAST('1152921504606846976' AS JSONB)""" + qt_select """SELECT CAST('6.18' AS JSONB)""" + qt_select """SELECT CAST('"abcd"' AS JSONB)""" + qt_select """SELECT CAST('{}' AS JSONB)""" + qt_select """SELECT CAST('{"k1":"v31", "k2": 300}' AS JSONB)""" + qt_select """SELECT CAST('[]' AS JSONB)""" + qt_select """SELECT CAST('[123, 456]' AS JSONB)""" + qt_select """SELECT CAST('["abc", "def"]' AS JSONB)""" + qt_select """SELECT CAST('[null, true, false, 100, 6.18, "abc"]' AS JSONB)""" + qt_select """SELECT CAST('[{"k1":"v41", "k2": 400}, 1, "a", 3.14]' AS JSONB)""" + qt_select """SELECT CAST('{"k1":"v31", "k2": 300, "a1": [{"k1":"v41", "k2": 400}, 1, "a", 3.14]}' AS JSONB)""" + qt_select """SELECT CAST("''" AS JSONB)""" + qt_select """SELECT CAST("'abc'" AS JSONB)""" + qt_select """SELECT CAST('abc' AS JSONB)""" + qt_select """SELECT CAST('100x' AS JSONB)""" + qt_select """SELECT CAST('6.a8' AS JSONB)""" + qt_select """SELECT CAST('{x' AS JSONB)""" + qt_select """SELECT CAST('[123, abc]' AS JSONB)""" + +} diff --git a/regression-test/suites/nereids_syntax_p0/information_schema.groovy b/regression-test/suites/nereids_syntax_p0/information_schema.groovy index c4fead201798d5..59ab91ab97d9ef 100644 --- a/regression-test/suites/nereids_syntax_p0/information_schema.groovy +++ b/regression-test/suites/nereids_syntax_p0/information_schema.groovy @@ -18,7 +18,7 @@ suite("information_schema") { List> table = sql """ select * from backends(); """ assertTrue(table.size() > 0) - assertTrue(table[0].size == 24) + assertTrue(table[0].size == 25) sql "SELECT DATABASE();" sql "select USER();" diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf18.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf18.groovy index 541b2f6f8d3fa3..05c0fcb69d42cb 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf18.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf18.groovy @@ -84,5 +84,5 @@ suite("ds_rf18") { // File file = new File(outFile) // file.write(getRuntimeFilters(plan)) - assertEquals("RF5[c_current_cdemo_sk->[cd_demo_sk],RF4[cs_item_sk->[i_item_sk],RF3[ca_address_sk->[c_current_addr_sk],RF2[c_customer_sk->[cs_bill_customer_sk],RF1[d_date_sk->[cs_sold_date_sk],RF0[cd_demo_sk->[cs_bill_cdemo_sk]", getRuntimeFilters(plan)) + assertEquals("RF5[cs_item_sk->[i_item_sk],RF4[c_current_cdemo_sk->[cd_demo_sk],RF3[ca_address_sk->[c_current_addr_sk],RF2[c_customer_sk->[cs_bill_customer_sk],RF1[d_date_sk->[cs_sold_date_sk],RF0[cd_demo_sk->[cs_bill_cdemo_sk]", getRuntimeFilters(plan)) } diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf59.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf59.groovy index e97af073f25d2a..af6fea1f90cc61 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf59.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf59.groovy @@ -93,5 +93,5 @@ limit 100; // def outFile = "regression-test/suites/nereids_tpcds_shape_sf100_p0/ddl/rf/rf.59" // File file = new File(outFile) // file.write(getRuntimeFilters(plan)) - assertEquals("RF0[d_date_sk->[ss_sold_date_sk],RF4[s_store_id2->[s_store_id],RF5[s_store_sk->[ss_store_sk],RF3[s_store_sk->[ss_store_sk],RF2[d_week_seq->[d_week_seq],RF1[d_week_seq->[d_week_seq]", getRuntimeFilters(plan)) + assertEquals("RF0[d_date_sk->[ss_sold_date_sk],RF4[s_store_sk->[ss_store_sk],RF3[d_week_seq->[d_week_seq],RF2[s_store_sk->[ss_store_sk],RF1[d_week_seq->[d_week_seq]", getRuntimeFilters(plan)) } diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf61.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf61.groovy index da153785b44a37..f567f8c0775012 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf61.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf61.groovy @@ -94,5 +94,5 @@ limit 100; // File file = new File(outFile) // file.write(getRuntimeFilters(plan)) - assertEquals("RF10[ss_item_sk->[i_item_sk],RF9[c_current_addr_sk->[ca_address_sk],RF8[ss_customer_sk->[c_customer_sk],RF7[p_promo_sk->[ss_promo_sk],RF6[d_date_sk->[ss_sold_date_sk],RF5[s_store_sk->[ss_store_sk],RF4[ss_item_sk->[i_item_sk],RF3[c_current_addr_sk->[ca_address_sk],RF2[ss_customer_sk->[c_customer_sk],RF1[d_date_sk->[ss_sold_date_sk],RF0[s_store_sk->[ss_store_sk]", getRuntimeFilters(plan)) + assertEquals("RF10[ss_item_sk->[i_item_sk],RF9[c_current_addr_sk->[ca_address_sk],RF8[ss_customer_sk->[c_customer_sk],RF7[d_date_sk->[ss_sold_date_sk],RF6[p_promo_sk->[ss_promo_sk],RF5[s_store_sk->[ss_store_sk],RF4[ss_item_sk->[i_item_sk],RF3[c_current_addr_sk->[ca_address_sk],RF2[ss_customer_sk->[c_customer_sk],RF1[d_date_sk->[ss_sold_date_sk],RF0[s_store_sk->[ss_store_sk]", getRuntimeFilters(plan)) } diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf72.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf72.groovy index a17c335617bae1..90789b117f7c4e 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf72.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf72.groovy @@ -79,5 +79,5 @@ limit 100; // File file = new File(outFile) // file.write(getRuntimeFilters(plan)) - assertEquals("RF8[w_warehouse_sk->[inv_warehouse_sk],RF6[d_week_seq->[d_week_seq],RF7[d_date_sk->[inv_date_sk],RF5[cs_item_sk->[inv_item_sk],RF4[i_item_sk->[cs_item_sk],RF3[d_date_sk->[cs_ship_date_sk],RF2[cd_demo_sk->[cs_bill_cdemo_sk],RF1[d_date_sk->[cs_sold_date_sk],RF0[hd_demo_sk->[cs_bill_hdemo_sk]", getRuntimeFilters(plan)) + assertEquals("RF8[w_warehouse_sk->[inv_warehouse_sk],RF6[d_date_sk->[inv_date_sk],RF7[cs_item_sk->[inv_item_sk],RF5[d_week_seq->[d_week_seq],RF4[i_item_sk->[cs_item_sk],RF3[d_date_sk->[cs_ship_date_sk],RF2[cd_demo_sk->[cs_bill_cdemo_sk],RF1[d_date_sk->[cs_sold_date_sk],RF0[hd_demo_sk->[cs_bill_hdemo_sk]", getRuntimeFilters(plan)) } diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf99.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf99.groovy index 4093147f822886..4f627fb721552d 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf99.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/rf/ds_rf99.groovy @@ -85,5 +85,5 @@ limit 100; // File file = new File(outFile) // file.write(getRuntimeFilters(plan)) - assertEquals("RF3[w_warehouse_sk->[cs_warehouse_sk],RF2[sm_ship_mode_sk->[cs_ship_mode_sk],RF1[cc_call_center_sk->[cs_call_center_sk],RF0[d_date_sk->[cs_ship_date_sk]", getRuntimeFilters(plan)) + assertEquals("RF3[cc_call_center_sk->[cs_call_center_sk],RF2[sm_ship_mode_sk->[cs_ship_mode_sk],RF1[w_warehouse_sk->[cs_warehouse_sk],RF0[d_date_sk->[cs_ship_date_sk]", getRuntimeFilters(plan)) } diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf8.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf8.groovy index f223af342adc0f..76d0eec9938694 100644 --- a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf8.groovy +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf/h_rf8.groovy @@ -105,5 +105,5 @@ order by // def outFile = "regression-test/suites/nereids_tpch_shape_sf1000_p0/ddl/rf/rf.8" // File file = new File(outFile) // file.write(getRuntimeFilters(plan)) - assertEquals("RF6[n_nationkey->[s_nationkey],RF5[l_suppkey->[s_suppkey],RF4[r_regionkey->[n_regionkey],RF3[n_nationkey->[c_nationkey],RF2[o_custkey->[c_custkey],RF1[l_orderkey->[o_orderkey],RF0[p_partkey->[l_partkey]", getRuntimeFilters(plan)) + assertEquals("RF6[n_nationkey->[s_nationkey],RF5[l_suppkey->[s_suppkey],RF4[l_orderkey->[o_orderkey],RF3[c_custkey->[o_custkey],RF2[n_nationkey->[c_nationkey],RF1[r_regionkey->[n_regionkey],RF0[p_partkey->[l_partkey]", getRuntimeFilters(plan)) } diff --git a/regression-test/suites/partition_p0/auto_partition/test_auto_list_partition.groovy b/regression-test/suites/partition_p0/auto_partition/test_auto_list_partition.groovy index 405b2a1fc5cf77..20f07156da3b34 100644 --- a/regression-test/suites/partition_p0/auto_partition/test_auto_list_partition.groovy +++ b/regression-test/suites/partition_p0/auto_partition/test_auto_list_partition.groovy @@ -88,4 +88,86 @@ suite("test_auto_list_partition") { result3 = sql "show partitions from ${tblName3}" logger.info("${result3}") assertEquals(result3.size(), 4) + + def tblName4 = "list_table4" + sql "drop table if exists ${tblName4}" + sql """ + CREATE TABLE `${tblName4}` ( + `k1` INT not null, + `k2` VARCHAR(50), + `k3` DATETIMEV2(6) + ) ENGINE=OLAP + DUPLICATE KEY(`k1`) + COMMENT 'OLAP' + AUTO PARTITION BY LIST (`k1`) + ( + ) + DISTRIBUTED BY HASH(`k1`) BUCKETS 16 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + sql """ insert into ${tblName4} values (1, '2020-12-12 12:12:12', '2000-01-01 12:12:12.123456'), (2, '20201212 121212', '2000-01-01'), (3, '20201212121212', '2000-01-01'), (3, 'AaA', '2000-01-01') """ + result4 = sql "show partitions from ${tblName4}" + logger.info("${result4}") + assertEquals(result4.size(), 3) + + def tblName5 = "list_table4" + sql "drop table if exists ${tblName5}" + sql """ + CREATE TABLE `${tblName5}` ( + `k1` INT, + `k2` VARCHAR(50), + `k3` DATETIMEV2(6) not null + ) ENGINE=OLAP + DUPLICATE KEY(`k1`) + COMMENT 'OLAP' + AUTO PARTITION BY LIST (`k3`) + ( + ) + DISTRIBUTED BY HASH(`k1`) BUCKETS 16 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + sql """ insert into ${tblName5} values (1, 'ABC', '2000-01-01 12:12:12.123456'), (2, 'AAA', '2000-01-01'), (3, 'aaa', '2000-01-01'), (3, 'AaA', '2000-01-01') """ + result5 = sql "show partitions from ${tblName5}" + logger.info("${result5}") + assertEquals(result5.size(), 2) + + sql "drop table if exists test_largeint" + sql """ + CREATE TABLE test_largeint ( + k largeint not null + ) + AUTO PARTITION BY LIST (`k`) + ( + ) + DISTRIBUTED BY HASH(`k`) BUCKETS 16 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + sql " insert into test_largeint values (1), (-1)" + result6 = sql "show partitions from test_largeint" + logger.info("${result6}") + assertEquals(result6.size(), 2) + + sql "drop table if exists test_bool" + sql """ + CREATE TABLE test_bool ( + k boolean not null + ) + AUTO PARTITION BY LIST (`k`) + ( + ) + DISTRIBUTED BY HASH(`k`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + sql " insert into test_bool values (true), (false)" + result7 = sql "show partitions from test_bool" + logger.info("${result7}") + assertEquals(result7.size(), 2) } diff --git a/regression-test/suites/partition_p0/test_partition_operation.groovy b/regression-test/suites/partition_p0/test_partition_operation.groovy index 05842a6532cbd7..6c9893d23c1c10 100644 --- a/regression-test/suites/partition_p0/test_partition_operation.groovy +++ b/regression-test/suites/partition_p0/test_partition_operation.groovy @@ -154,9 +154,8 @@ suite("test_partition_operation", "p1") { checkTablePartitionNotExists("test_drop_partition_1", "partition_a") sql """ALTER TABLE test_drop_partition_1 DROP PARTITION partition_b""" checkTablePartitionNotExists("test_drop_partition_1", "partition_b") - sql """ALTER TABLE test_drop_partition_1 DROP PARTITION partition_c""" + sql """ALTER TABLE test_drop_partition_1 DROP PARTITION partition_c, DROP PARTITION partition_d""" checkTablePartitionNotExists("test_drop_partition_1", "partition_c") - sql """ALTER TABLE test_drop_partition_1 DROP PARTITION partition_d""" checkTablePartitionNotExists("test_drop_partition_1", "partition_d") qt_sql5 "select * from test_drop_partition_1 order by k1, k2" // after drop all partition, add a partiion diff --git a/regression-test/suites/pipelineX/test_distinct_streaming_agg_operator.groovy b/regression-test/suites/pipelineX/test_distinct_streaming_agg_operator.groovy index f0337fcfc7f346..a8fc2d5868badf 100644 --- a/regression-test/suites/pipelineX/test_distinct_streaming_agg_operator.groovy +++ b/regression-test/suites/pipelineX/test_distinct_streaming_agg_operator.groovy @@ -61,7 +61,7 @@ suite("test_distinct_streaming_agg_operator") { select k6 from baseall group by k6 order by k6; """ - sql"""set experimental_enable_pipeline_x_engine=true; """ + sql"""set experimental_enable_pipeline_x_engine=false; """ qt_pipelineX_1 """ select * from ( select k1,k2,k3 from baseall union select k1,k2,k3 from baseall) as t ORDER BY 1, 2,3; diff --git a/regression-test/suites/pipelineX/test_partition_sort_operator.groovy b/regression-test/suites/pipelineX/test_partition_sort_operator.groovy index 22fe5bc20c3e0e..73a3d8cc749ce3 100644 --- a/regression-test/suites/pipelineX/test_partition_sort_operator.groovy +++ b/regression-test/suites/pipelineX/test_partition_sort_operator.groovy @@ -64,7 +64,7 @@ suite("test_partition_sort_operator") { ORDER BY 1, 2,3; """ - sql"""set experimental_enable_pipeline_x_engine=true; """ + sql"""set experimental_enable_pipeline_x_engine=false; """ qt_pipelineX_1 """ select * from (select k6,k2,row_number() over(partition by k6 order by k2) as num from baseall) as res where num < 5 diff --git a/regression-test/suites/pipelineX/test_repeat_operator.groovy b/regression-test/suites/pipelineX/test_repeat_operator.groovy index 57357b0b4a6b3e..cb164aab0983c8 100644 --- a/regression-test/suites/pipelineX/test_repeat_operator.groovy +++ b/regression-test/suites/pipelineX/test_repeat_operator.groovy @@ -70,7 +70,7 @@ suite("test_repeat_operator") { ORDER BY k1, k2,k3; """ - sql"""set experimental_enable_pipeline_x_engine=true; """ + sql"""set experimental_enable_pipeline_x_engine=false; """ qt_pipelineX """ SELECT k1, k2 diff --git a/regression-test/suites/pipelineX/test_union_operator.groovy b/regression-test/suites/pipelineX/test_union_operator.groovy index 59448c35ec6a40..7f606cfa7f57a8 100644 --- a/regression-test/suites/pipelineX/test_union_operator.groovy +++ b/regression-test/suites/pipelineX/test_union_operator.groovy @@ -80,7 +80,7 @@ suite("test_union_operator") { """ - sql"""set experimental_enable_pipeline_x_engine=true,parallel_pipeline_task_num = 8;; """ + sql"""set experimental_enable_pipeline_x_engine=false,parallel_pipeline_task_num = 8;; """ qt_pipelineX """ SELECT count(*) diff --git a/regression-test/suites/query_p0/sql_functions/array_functions/test_array_functions.groovy b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_functions.groovy index 6a770daa799a3c..c043ada07af502 100644 --- a/regression-test/suites/query_p0/sql_functions/array_functions/test_array_functions.groovy +++ b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_functions.groovy @@ -34,7 +34,8 @@ suite("test_array_functions") { `k10` ARRAY NULL COMMENT "", `k11` ARRAY NULL COMMENT "", `k12` ARRAY NULL COMMENT "", - `k13` ARRAY NULL COMMENT "" + `k13` ARRAY NULL COMMENT "", + `k14` ARRAY NULL COMMENT "" ) ENGINE=OLAP DUPLICATE KEY(`k1`) DISTRIBUTED BY HASH(`k1`) BUCKETS 1 @@ -43,15 +44,15 @@ suite("test_array_functions") { "storage_format" = "V2" ) """ - sql """ INSERT INTO ${tableName} VALUES(1,[1,2,3],["a","b",""],[1,2],["hi"],["2015-03-13"],["2015-03-13 12:36:38"],["2023-02-05","2023-02-06"],["2023-02-07","2023-02-06"],['2022-10-15 10:30:00.999', '2022-08-31 12:00:00.999'],['2022-10-16 10:30:00.999', '2022-08-31 12:00:00.999'],[111.111, 222.222],[222.222, 333.333]) """ - sql """ INSERT INTO ${tableName} VALUES(2,[4],NULL,[5],["hi2"],NULL,NULL,["2023-01-05","2023-01-06"],["2023-01-07","2023-01-06"],['2022-11-15 10:30:00.999', '2022-01-31 12:00:00.999'],['2022-11-16 10:30:00.999', '2022-01-31 12:00:00.999'],[333.3333, 444.4444],[444.4444, 555.5555]) """ - sql """ INSERT INTO ${tableName} VALUES(3,[],[],NULL,["hi3"],NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL) """ - sql """ INSERT INTO ${tableName} VALUES(4,[1,2,3,4,5,4,3,2,1],[],[],NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL) """ - sql """ INSERT INTO ${tableName} VALUES(5,[],["a","b","c","d","c","b","a"],NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL) """ - sql """ INSERT INTO ${tableName} VALUES(6,[1,2,3,4,5,4,3,2,1],["a","b","c","d","c","b","a"],NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL) """ - sql """ INSERT INTO ${tableName} VALUES(7,[8,9,NULL,10,NULL],["f",NULL,"g",NULL,"h"],NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL) """ - sql """ INSERT INTO ${tableName} VALUES(8,[1,2,3,3,4,4,NULL],["a","b","b","b"],[1,2,2,3],["hi","hi","hello"],["2015-03-13"],["2015-03-13 12:36:38"],NULL,NULL,NULL,NULL,NULL,NULL) """ - sql """ INSERT INTO ${tableName} VALUES(9,[1,2,3],["a","b",""],[1,2],["hi"],["2015-03-13","2015-03-13","2015-03-14"],["2015-03-13 12:36:38","2015-03-13 12:36:38"],NULL,NULL,NULL,NULL,NULL,NULL) """ + sql """ INSERT INTO ${tableName} VALUES(1,[1,2,3],["a","b",""],[1,2],["hi"],["2015-03-13"],["2015-03-13 12:36:38"],["2023-02-05","2023-02-06"],["2023-02-07","2023-02-06"],['2022-10-15 10:30:00.999', '2022-08-31 12:00:00.999'],['2022-10-16 10:30:00.999', '2022-08-31 12:00:00.999'],[111.111, 222.222],[222.222, 333.333],[1,222,3]) """ + sql """ INSERT INTO ${tableName} VALUES(2,[4],NULL,[5],["hi2"],NULL,NULL,["2023-01-05","2023-01-06"],["2023-01-07","2023-01-06"],['2022-11-15 10:30:00.999', '2022-01-31 12:00:00.999'],['2022-11-16 10:30:00.999', '2022-01-31 12:00:00.999'],[333.3333, 444.4444],[444.4444, 555.5555],NULL) """ + sql """ INSERT INTO ${tableName} VALUES(3,[],[],NULL,["hi3"],NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,[2,3,4]) """ + sql """ INSERT INTO ${tableName} VALUES(4,[1,2,3,4,5,4,3,2,1],[],[],NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,[NULL, 23]) """ + sql """ INSERT INTO ${tableName} VALUES(5,[],["a","b","c","d","c","b","a"],NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,[]) """ + sql """ INSERT INTO ${tableName} VALUES(6,[1,2,3,4,5,4,3,2,1],["a","b","c","d","c","b","a"],NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,[NULL, 22]) """ + sql """ INSERT INTO ${tableName} VALUES(7,[8,9,NULL,10,NULL],["f",NULL,"g",NULL,"h"],NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,[8,9,NULL,10,NULL]) """ + sql """ INSERT INTO ${tableName} VALUES(8,[1,2,3,3,4,4,NULL],["a","b","b","b"],[1,2,2,3],["hi","hi","hello"],["2015-03-13"],["2015-03-13 12:36:38"],NULL,NULL,NULL,NULL,NULL,NULL,[8,9,NULL,10,NULL]) """ + sql """ INSERT INTO ${tableName} VALUES(9,[1,2,3],["a","b",""],[1,2],["hi"],["2015-03-13","2015-03-13","2015-03-14"],["2015-03-13 12:36:38","2015-03-13 12:36:38"],NULL,NULL,NULL,NULL,NULL,NULL,[NULL,12]) """ qt_select "SELECT k1, size(k2), size(k3) FROM ${tableName} ORDER BY k1" qt_select "SELECT k1, cardinality(k2), cardinality(k3) FROM ${tableName} ORDER BY k1" @@ -76,6 +77,8 @@ suite("test_array_functions") { qt_select "SELECT k1, array_union(k8, k9) FROM ${tableName} ORDER BY k1" qt_select "SELECT k1, array_union(k10, k11) FROM ${tableName} ORDER BY k1" qt_select "SELECT k1, array_union(k12, k13) FROM ${tableName} ORDER BY k1" + // multi-params array_union + qt_select "SELECT k1, array_union(k2, k4, k14) FROM ${tableName} ORDER BY k1" qt_select "SELECT k1, array_except(k2, k4) FROM ${tableName} ORDER BY k1" qt_select "SELECT k1, array_except(k8, k9) FROM ${tableName} ORDER BY k1" qt_select "SELECT k1, array_except(k10, k11) FROM ${tableName} ORDER BY k1" diff --git a/regression-test/suites/query_p0/sql_functions/array_functions/test_array_functions_by_literal.groovy b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_functions_by_literal.groovy index f35549019e5aac..58e655608a5c79 100644 --- a/regression-test/suites/query_p0/sql_functions/array_functions/test_array_functions_by_literal.groovy +++ b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_functions_by_literal.groovy @@ -248,6 +248,12 @@ suite("test_array_functions_by_literal") { qt_sql_intersect_3 "select array_intersect([1,2,3, null], [1,2,3,null], [1,2,null], [1, null])" qt_sql_intersect_4 "select array_intersect([1,2,3], [1,2,3], [null], [])" + // array_union-with-multiple-arguments + qt_sql_union_1 "select array_union([1,2,3], [1,2,3], [null])" + qt_sql_union_2 "select array_union([1, 2, null], [1, 3, null], [1,2,3,null])" + qt_sql_union_3 "select array_union([1,2,3, null], [1,2,3,null], [1,2,null], [1, null])" + qt_sql_union_4 "select array_union([1,2,3], [1,2,3], [null], [])" + // array_popfront function qt_sql "select array_popfront([1,2,3,4,5,6])" qt_sql "select array_popfront([])" diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.1.sql index c3d19b67a2889e..67aa90696de289 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE LO_ORDERDATE >= 19930101 diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.2.sql index 6ab6ceea340c88..d3d9b16cc7bbe8 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q1.2 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE LO_ORDERDATE >= 19940101 diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.3.sql index 70796c2a954cff..b22efea51b01ef 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q1.3 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE weekofyear(LO_ORDERDATE) = 6 diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.1.sql index 57f2ada2964909..7dbab32f964f31 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q2.1 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, P_BRAND FROM lineorder_flat diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.2.sql index 9b7a5db502f6be..884a7288fd3179 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q2.2 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, P_BRAND FROM lineorder_flat diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.3.sql index 3a8a5e74d48f7f..7180f48a4a41ef 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q2.3 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, P_BRAND FROM lineorder_flat diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.1.sql index 6b3257f1f3b239..927e504328a72b 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q3.1 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ C_NATION, S_NATION, (LO_ORDERDATE DIV 10000) AS YEAR, SUM(LO_REVENUE) AS revenue diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.2.sql index fefe727da82d18..63ac7953926c4b 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q3.2 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ C_CITY, S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, SUM(LO_REVENUE) AS revenue diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.3.sql index c4560b701edfdb..25430249b0bf2e 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q3.3 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ C_CITY, S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, SUM(LO_REVENUE) AS revenue diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.4.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.4.sql index 4ae5d956e48e17..de0750590096a9 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.4.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.4.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q3.4 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ C_CITY, S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, SUM(LO_REVENUE) AS revenue diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.1.sql index 87b29bf160acde..263b9e55e4da32 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q4.1 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ (LO_ORDERDATE DIV 10000) AS YEAR, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ (LO_ORDERDATE DIV 10000) AS YEAR, C_NATION, SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit FROM lineorder_flat diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.2.sql index 8ea28f3f120d5f..587eeef2622ae3 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q4.2 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ (LO_ORDERDATE DIV 10000) AS YEAR, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ (LO_ORDERDATE DIV 10000) AS YEAR, S_NATION, P_CATEGORY, SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.3.sql index 0f7c7401ab630e..fadb807c0b01fd 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q4.3 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ (LO_ORDERDATE DIV 10000) AS YEAR, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ (LO_ORDERDATE DIV 10000) AS YEAR, S_CITY, P_BRAND, SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.1.sql index 50b50bc3683925..934377807c60eb 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_extendedprice*lo_discount) AS +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(lo_extendedprice*lo_discount) AS REVENUE FROM lineorder, date WHERE lo_orderdate = d_datekey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.2.sql index 77c02620167681..a86974c8a1b4f8 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_extendedprice*lo_discount) AS +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(lo_extendedprice*lo_discount) AS REVENUE FROM lineorder, date WHERE lo_orderdate = d_datekey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.3.sql index 0052db0aac3460..41ac021e963856 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_extendedprice*lo_discount) AS +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(lo_extendedprice*lo_discount) AS REVENUE FROM lineorder, date WHERE lo_orderdate = d_datekey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.1.sql index a47ec82b517028..b4777810370e04 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_revenue), d_year, p_brand +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(lo_revenue), d_year, p_brand FROM lineorder, date, part, supplier WHERE lo_orderdate = d_datekey AND lo_partkey = p_partkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.2.sql index 9ab1a95d4d73d2..fc95bc2d87a367 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_revenue), d_year, p_brand +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(lo_revenue), d_year, p_brand FROM lineorder, date, part, supplier WHERE lo_orderdate = d_datekey AND lo_partkey = p_partkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.3.sql index b7e6bd784098ec..20f6baf359ebd4 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_revenue), d_year, p_brand +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(lo_revenue), d_year, p_brand FROM lineorder, date, part, supplier WHERE lo_orderdate = d_datekey AND lo_partkey = p_partkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.1.sql index 85c470b70860e5..daead15361251e 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_nation, s_nation, d_year, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ c_nation, s_nation, d_year, SUM(lo_revenue) AS REVENUE FROM customer, lineorder, supplier, date WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.2.sql index cd0b320f87ff9a..1b729ea6f8c42c 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_city, s_city, d_year, sum(lo_revenue) +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ c_city, s_city, d_year, sum(lo_revenue) AS REVENUE FROM customer, lineorder, supplier, date WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.3.sql index 89765c02d9c6c1..90450d3729bfb3 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_city, s_city, d_year, SUM(lo_revenue) +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ c_city, s_city, d_year, SUM(lo_revenue) AS REVENUE FROM customer, lineorder, supplier, date WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.4.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.4.sql index 5cef87a3fe3350..c59e9f10b03d62 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.4.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.4.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_city, s_city, d_year, SUM(lo_revenue) +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ c_city, s_city, d_year, SUM(lo_revenue) AS REVENUE FROM customer, lineorder, supplier, date WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.1.sql index 3e0227c2ea96f2..767af86aba2d55 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ d_year, c_nation, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ d_year, c_nation, SUM(lo_revenue - lo_supplycost) AS PROFIT FROM date, customer, supplier, part, lineorder WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.2.sql index 1338e780ae52ca..8accfc8e5a44de 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ d_year, s_nation, p_category, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ d_year, s_nation, p_category, SUM(lo_revenue - lo_supplycost) AS PROFIT FROM date, customer, supplier, part, lineorder WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.3.sql index d8e6f7c42d4074..f5411faaa42707 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ d_year, s_city, p_brand, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ d_year, s_city, p_brand, SUM(lo_revenue - lo_supplycost) AS PROFIT FROM date, customer, supplier, part, lineorder WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q01.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q01.sql index ded6754a971c20..78b560ff246d24 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q01.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q01.sql @@ -1,5 +1,5 @@ -- tables: lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ l_returnflag, l_linestatus, sum(l_quantity) AS sum_qty, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q02.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q02.sql index f102f7504d7f82..510c3d117ddec8 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q02.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q02.sql @@ -1,5 +1,5 @@ -- tables: part,supplier,partsupp,nation,region -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ s_acctbal, s_name, n_name, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q03.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q03.sql index 8bd60f0e0799b5..360e51763fdcf6 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q03.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q03.sql @@ -1,5 +1,5 @@ -- tables: customer,orders,lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ l_orderkey, sum(l_extendedprice * (1 - l_discount)) AS revenue, o_orderdate, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q04.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q04.sql index 3f440947290b90..99a1213d5c6f7b 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q04.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q04.sql @@ -1,5 +1,5 @@ -- tables: orders,lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ o_orderpriority, count(*) AS order_count FROM orders diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q05.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q05.sql index ed179f8b869be8..871c3cc84c26ce 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q05.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q05.sql @@ -1,5 +1,5 @@ -- tables: customer,orders,lineitem,supplier,nation,region -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ n_name, sum(l_extendedprice * (1 - l_discount)) AS revenue FROM diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q06.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q06.sql index 2dd86f8c2ce55e..b08a070250ac56 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q06.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q06.sql @@ -1,6 +1,6 @@ -- tables: lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ sum(l_extendedprice * l_discount) AS revenue +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ sum(l_extendedprice * l_discount) AS revenue FROM lineitem WHERE diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q07.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q07.sql index 6453c1094a836d..77c13aa4e8cacc 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q07.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q07.sql @@ -1,5 +1,5 @@ -- tables: supplier,lineitem,orders,customer,nation -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ supp_nation, cust_nation, l_year, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q08.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q08.sql index e4c46fb0847bba..bbdb4fea5822e5 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q08.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q08.sql @@ -1,5 +1,5 @@ -- tables: part,supplier,lineitem,orders,customer,nation,region -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ o_year, sum(CASE WHEN nation = 'BRAZIL' diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q09.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q09.sql index cee9925fb5d902..0e7e1a63262f7c 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q09.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q09.sql @@ -1,5 +1,5 @@ -- tables: part,supplier,lineitem,partsupp,orders,nation -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ nation, o_year, sum(amount) AS sum_profit diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q10.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q10.sql index c95a80fceef4fe..2a78f2931f6f7c 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q10.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q10.sql @@ -1,5 +1,5 @@ -- tables: customer,orders,lineitem,nation -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ c_custkey, c_name, sum(l_extendedprice * (1 - l_discount)) AS revenue, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q11.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q11.sql index b23701e940722e..eb0987288a7874 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q11.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q11.sql @@ -1,5 +1,5 @@ -- tables: partsupp,supplier,nation -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ ps_partkey, sum(ps_supplycost * ps_availqty) AS value FROM diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q12.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q12.sql index e8893e71e4eae3..d72e552a339657 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q12.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q12.sql @@ -1,5 +1,5 @@ -- tables: orders,lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ l_shipmode, sum(CASE WHEN o_orderpriority = '1-URGENT' diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q13.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q13.sql index 9db2da60ee4388..d222de86947098 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q13.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q13.sql @@ -1,5 +1,5 @@ -- tables: customer -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ c_count, count(*) AS custdist FROM ( diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q14.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q14.sql index 70d7a57d07dcbc..b70a8abfd8a1fb 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q14.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q14.sql @@ -1,5 +1,5 @@ -- tables: lineitem,part -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ 100.00 * sum(CASE +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ 100.00 * sum(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1 - l_discount) ELSE 0 diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q15.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q15.sql index 45f75ff985778b..fd526df87b088c 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q15.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q15.sql @@ -1,4 +1,4 @@ -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ s_suppkey, s_name, s_address, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q16.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q16.sql index 37a438c796129e..c21c8fb494655e 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q16.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q16.sql @@ -1,5 +1,5 @@ -- tables: partsupp,part,supplier -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ p_brand, p_type, p_size, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q17.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q17.sql index 62f39a750c2c07..ae11f9c699d236 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q17.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q17.sql @@ -1,5 +1,5 @@ -- tables: lineitem,part -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ sum(l_extendedprice) / 7.0 AS avg_yearly +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ sum(l_extendedprice) / 7.0 AS avg_yearly FROM lineitem, part diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q18.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q18.sql index 2eb2505c01e75a..910b9ffc7f71ff 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q18.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q18.sql @@ -1,5 +1,5 @@ -- tables: customer,orders,lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ c_name, c_custkey, o_orderkey, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q19.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q19.sql index 16e543f87c0eda..b55400e4ae5fc8 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q19.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q19.sql @@ -1,5 +1,5 @@ -- tables: lineitem,part -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ sum(l_extendedprice * (1 - l_discount)) AS revenue +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ sum(l_extendedprice * (1 - l_discount)) AS revenue FROM lineitem, part diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q20.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q20.sql index a2aca567901752..550c00b4e40d05 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q20.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q20.sql @@ -1,5 +1,5 @@ -- tables: supplier,nation,partsupp,lineitem,part -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ s_name, s_address FROM diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q21.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q21.sql index 7b4874f96c7431..099b5ad10657e3 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q21.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q21.sql @@ -1,5 +1,5 @@ -- tables: supplier,lineitem,orders,nation -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ s_name, count(*) AS numwait FROM diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q22.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q22.sql index bf784175e0ed2d..fb0d71a6bc153c 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q22.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q22.sql @@ -1,5 +1,5 @@ -- tables: orders,customer -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ cntrycode, count(*) AS numcust, sum(c_acctbal) AS totacctbal diff --git a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_native_insert_stmt.groovy b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_native_insert_stmt.groovy index 23a44c846e09a1..ce9b425ff9db62 100644 --- a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_native_insert_stmt.groovy +++ b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_native_insert_stmt.groovy @@ -100,6 +100,7 @@ suite("test_partial_update_native_insert_stmt", "p0") { sql """insert into ${tableName3} values(2, "doris2", 2000, 223, 1),(1, "doris", 1000, 123, 1);""" qt_3 """ select * from ${tableName3} order by id; """ sql "set enable_unique_key_partial_update=true;" + sql "set enable_insert_strict = false;" sql "sync;" // in partial update, the unmentioned columns should have default values or be nullable // but field `name` is not nullable and doesn't have default value diff --git a/sonar-scanner.properties b/sonar-project.properties similarity index 100% rename from sonar-scanner.properties rename to sonar-project.properties